Add SyncSegments into proto (#19312)

See also: #19072

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

Signed-off-by: yangxuan <xuan.yang@zilliz.com>
This commit is contained in:
XuanYang-cn 2022-09-23 10:22:52 +08:00 committed by GitHub
parent 1c489a85b2
commit 432b5ddf2e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 872 additions and 324 deletions

View File

@ -244,6 +244,10 @@ func (c *mockDataNodeClient) AddSegment(ctx context.Context, req *datapb.AddSegm
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) Stop() error {
c.state = internalpb.StateCode_Abnormal
return nil

View File

@ -877,6 +877,65 @@ func (node *DataNode) GetCompactionState(ctx context.Context, req *datapb.Compac
}, nil
}
// SyncSegments called by DataCoord, sync the compacted segments' meta between DC and DN
func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
log.Ctx(ctx).Info("DataNode receives SyncSegments",
zap.Int64("planID", req.GetPlanID()),
zap.Int64("target segmentID", req.GetCompactedTo()),
zap.Int64s("compacted from", req.GetCompactedFrom()),
zap.Int64("numOfRows", req.GetNumOfRows()),
)
status := &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}
if !node.isHealthy() {
status.Reason = "DataNode is unhealthy"
return status, nil
}
if len(req.GetCompactedFrom()) <= 0 {
status.Reason = "invalid request, compacted from segments shouldn't be empty"
return status, nil
}
oneSegment := req.GetCompactedFrom()[0]
replica, err := node.flowgraphManager.getReplica(oneSegment)
if err != nil {
status.Reason = fmt.Sprintf("invalid request, err=%s", err.Error())
return status, nil
}
// check if all compactedFrom segments are valid
var invalidSegIDs []UniqueID
for _, segID := range req.GetCompactedFrom() {
if !replica.hasSegment(segID, true) {
invalidSegIDs = append(invalidSegIDs, segID)
}
}
if len(invalidSegIDs) > 0 {
status.Reason = fmt.Sprintf("invalid request, some segments are not in the same replica: %v", invalidSegIDs)
return status, nil
}
// oneSegment is definitely in the replica, guaranteed by the check before.
collID, partID, _ := replica.getCollectionAndPartitionID(oneSegment)
chanName, _ := replica.getChannelName(oneSegment)
targetSeg := &Segment{
collectionID: collID,
partitionID: partID,
channelName: chanName,
segmentID: req.GetCompactedTo(),
numRows: req.GetNumOfRows(),
}
if err := replica.mergeFlushedSegments(targetSeg, req.GetPlanID(), req.GetCompactedFrom()); err != nil {
status.Reason = err.Error()
return status, nil
}
status.ErrorCode = commonpb.ErrorCode_Success
return status, nil
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
log.Info("DataNode receive import request",

View File

@ -604,6 +604,90 @@ func TestDataNode(t *testing.T) {
}
cancel()
})
t.Run("Test SyncSegments", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
chanName := "fake-by-dev-rootcoord-dml-test-syncsegments-1"
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err)
defer etcdCli.Close()
node.SetEtcdClient(etcdCli)
err = node.Init()
assert.Nil(t, err)
err = node.Start()
assert.Nil(t, err)
defer node.Stop()
err = node.flowgraphManager.addAndStart(node, &datapb.VchannelInfo{
ChannelName: chanName,
UnflushedSegmentIds: []int64{},
FlushedSegmentIds: []int64{},
})
require.NoError(t, err)
fg, ok := node.flowgraphManager.getFlowgraphService(chanName)
assert.True(t, ok)
fg.replica.(*SegmentReplica).flushedSegments = map[UniqueID]*Segment{
100: {channelName: chanName},
101: {channelName: chanName},
102: {channelName: chanName},
}
t.Run("invalid compacted from", func(t *testing.T) {
invalidCompactedFroms := [][]UniqueID{
{},
{100, 200},
}
req := &datapb.SyncSegmentsRequest{}
for _, invalid := range invalidCompactedFroms {
req.CompactedFrom = invalid
status, err := node.SyncSegments(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
}
})
t.Run("valid request numRows>0", func(t *testing.T) {
req := &datapb.SyncSegmentsRequest{
CompactedFrom: []int64{100, 101},
CompactedTo: 200,
NumOfRows: 100,
}
status, err := node.SyncSegments(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode())
assert.True(t, fg.replica.hasSegment(req.CompactedTo, true))
assert.False(t, fg.replica.hasSegment(req.CompactedFrom[0], true))
assert.False(t, fg.replica.hasSegment(req.CompactedFrom[1], true))
})
t.Run("valid request numRows=0", func(t *testing.T) {
fg.replica.(*SegmentReplica).flushedSegments = map[UniqueID]*Segment{
100: {channelName: chanName},
101: {channelName: chanName},
102: {channelName: chanName},
}
req := &datapb.SyncSegmentsRequest{
CompactedFrom: []int64{100, 101},
CompactedTo: 200,
NumOfRows: 0,
}
status, err := node.SyncSegments(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode())
assert.False(t, fg.replica.hasSegment(req.CompactedTo, true))
assert.False(t, fg.replica.hasSegment(req.CompactedFrom[0], true))
assert.False(t, fg.replica.hasSegment(req.CompactedFrom[1], true))
})
})
}
func TestDataNode_AddSegment(t *testing.T) {

View File

@ -211,7 +211,17 @@ func (dn *deleteNode) Operate(in []Msg) []Msg {
// update delBuf for compacted segments
func (dn *deleteNode) updateCompactedSegments() {
compactedTo2From := dn.replica.listCompactedSegmentIDs()
for compactedTo, compactedFrom := range compactedTo2From {
// if the compactedTo segment has 0 numRows, remove all segments related
if !dn.replica.hasSegment(compactedTo, true) {
for _, segID := range compactedFrom {
dn.delBuf.Delete(segID)
}
dn.replica.removeSegments(compactedFrom...)
continue
}
var compactToDelBuff *DelDataBuf
delBuf, loaded := dn.delBuf.Load(compactedTo)
if !loaded {
@ -223,8 +233,11 @@ func (dn *deleteNode) updateCompactedSegments() {
for _, segID := range compactedFrom {
if value, loaded := dn.delBuf.LoadAndDelete(segID); loaded {
compactToDelBuff.updateFromBuf(value.(*DelDataBuf))
dn.delBuf.Store(compactedTo, compactToDelBuff)
// only store delBuf if EntriesNum > 0
if compactToDelBuff.EntriesNum > 0 {
dn.delBuf.Store(compactedTo, compactToDelBuff)
}
}
}
log.Debug("update delBuf for compacted segments",

View File

@ -22,16 +22,16 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/bits-and-blooms/bloom/v3"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/api/schemapb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/milvus-io/milvus/internal/util/retry"
)
var deleteNodeTestDir = "/tmp/milvus_test/deleteNode"
@ -544,31 +544,41 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) {
require.NoError(t, err)
tests := []struct {
description string
segIDsInBuffer []UniqueID
description string
compactToExist bool
segIDsInBuffer []UniqueID
compactedToIDs []UniqueID
compactedFromIDs []UniqueID
expectedSegsRemain []UniqueID
}{
{"zero segments",
{"zero segments", false,
[]UniqueID{}, []UniqueID{}, []UniqueID{}, []UniqueID{}},
{"segment no compaction",
{"segment no compaction", false,
[]UniqueID{100, 101}, []UniqueID{}, []UniqueID{}, []UniqueID{100, 101}},
{"segment compacted not in buffer",
{"segment compacted not in buffer", true,
[]UniqueID{100, 101}, []UniqueID{200}, []UniqueID{103}, []UniqueID{100, 101}},
{"segment compacted in buffer one",
{"segment compacted in buffer 100>201", true,
[]UniqueID{100, 101}, []UniqueID{201}, []UniqueID{100}, []UniqueID{101, 201}},
{"segment compacted in buffer all-1",
{"segment compacted in buffer 100+101>201", true,
[]UniqueID{100, 101}, []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{201}},
{"segment compacted in buffer all-2",
{"segment compacted in buffer 100>201, 101>202", true,
[]UniqueID{100, 101}, []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{201, 202}},
// false
{"segment compacted in buffer 100>201", false,
[]UniqueID{100, 101}, []UniqueID{201}, []UniqueID{100}, []UniqueID{101}},
{"segment compacted in buffer 100+101>201", false,
[]UniqueID{100, 101}, []UniqueID{201, 201}, []UniqueID{100, 101}, []UniqueID{}},
{"segment compacted in buffer 100>201, 101>202", false,
[]UniqueID{100, 101}, []UniqueID{201, 202}, []UniqueID{100, 101}, []UniqueID{}},
}
for _, test := range tests {
t.Run(test.description, func(t *testing.T) {
for _, seg := range test.segIDsInBuffer {
delBuf := newDelDataBuf()
delBuf.updateSize(100)
delNode.delBuf.Store(seg, delBuf)
}
@ -579,20 +589,23 @@ func TestFlowGraphDeleteNode_updateCompactedSegments(t *testing.T) {
}
}
if test.compactToExist {
for _, seg := range test.compactedToIDs {
replica.flushedSegments[seg] = &Segment{
segmentID: seg,
numRows: 10,
}
}
} else {
replica.flushedSegments = make(map[UniqueID]*Segment)
}
delNode.updateCompactedSegments()
for _, remain := range test.expectedSegsRemain {
_, ok := delNode.delBuf.Load(remain)
assert.True(t, ok)
}
var count int
delNode.delBuf.Range(func(key, value interface{}) bool {
count++
return true
})
assert.Equal(t, len(test.expectedSegsRemain), count)
})
}
}

View File

@ -84,9 +84,7 @@ func (fm *flowgraphManager) release(vchanName string) {
}
func (fm *flowgraphManager) getFlushCh(segID UniqueID) (chan<- flushMsg, error) {
var (
flushCh chan flushMsg
)
var flushCh chan flushMsg
fm.flowgraphs.Range(func(key, value interface{}) bool {
fg := value.(*dataSyncService)
@ -104,6 +102,28 @@ func (fm *flowgraphManager) getFlushCh(segID UniqueID) (chan<- flushMsg, error)
return nil, fmt.Errorf("cannot find segment %d in all flowgraphs", segID)
}
func (fm *flowgraphManager) getReplica(segID UniqueID) (Replica, error) {
var (
rep Replica
exists = false
)
fm.flowgraphs.Range(func(key, value interface{}) bool {
fg := value.(*dataSyncService)
if fg.replica.hasSegment(segID, true) {
exists = true
rep = fg.replica
return false
}
return true
})
if exists {
return rep, nil
}
return nil, fmt.Errorf("cannot find segment %d in all flowgraphs", segID)
}
// resendTT loops through flow graphs, looks for segments that are not flushed, and sends them to that flow graph's
// `resendTTCh` channel so stats of these segments will be resent.
func (fm *flowgraphManager) resendTT() []UniqueID {

View File

@ -81,6 +81,47 @@ func TestFlowGraphManager(t *testing.T) {
fm.dropAll()
})
t.Run("Test getReplica", func(t *testing.T) {
vchanName := "by-dev-rootcoord-dml-test-flowgraphmanager-getReplica"
vchan := &datapb.VchannelInfo{
CollectionID: 1,
ChannelName: vchanName,
}
require.False(t, fm.exist(vchanName))
err := fm.addAndStart(node, vchan)
assert.NoError(t, err)
assert.True(t, fm.exist(vchanName))
fg, ok := fm.getFlowgraphService(vchanName)
require.True(t, ok)
err = fg.replica.addNewSegment(100, 1, 10, vchanName, &internalpb.MsgPosition{}, &internalpb.MsgPosition{})
require.NoError(t, err)
tests := []struct {
isvalid bool
inSegID UniqueID
description string
}{
{true, 100, "valid input for existed segmentID 100"},
{false, 101, "invalid input for not existed segmentID 101"},
}
for _, test := range tests {
t.Run(test.description, func(t *testing.T) {
rep, err := fm.getReplica(test.inSegID)
if test.isvalid {
assert.NoError(t, err)
assert.NotNil(t, rep)
} else {
assert.Error(t, err)
assert.Nil(t, rep)
}
})
}
})
t.Run("Test getFlushCh", func(t *testing.T) {
vchanName := "by-dev-rootcoord-dml-test-flowgraphmanager-getFlushCh"
vchan := &datapb.VchannelInfo{

View File

@ -57,6 +57,7 @@ type Replica interface {
getCollectionID() UniqueID
getCollectionSchema(collectionID UniqueID, ts Timestamp) (*schemapb.CollectionSchema, error)
getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error)
getChannelName(segID UniqueID) (string, error)
listAllSegmentIDs() []UniqueID
listNotFlushedSegmentIDs() []UniqueID
@ -273,6 +274,25 @@ func (replica *SegmentReplica) getCollectionAndPartitionID(segID UniqueID) (coll
return 0, 0, fmt.Errorf("cannot find segment, id = %v", segID)
}
func (replica *SegmentReplica) getChannelName(segID UniqueID) (string, error) {
replica.segMu.RLock()
defer replica.segMu.RUnlock()
if seg, ok := replica.newSegments[segID]; ok {
return seg.channelName, nil
}
if seg, ok := replica.normalSegments[segID]; ok {
return seg.channelName, nil
}
if seg, ok := replica.flushedSegments[segID]; ok {
return seg.channelName, nil
}
return "", fmt.Errorf("cannot find segment, id = %v", segID)
}
// maxRowCountPerSegment returns max row count for a segment based on estimation of row size.
func (replica *SegmentReplica) maxRowCountPerSegment(ts Timestamp) (int64, error) {
log := log.With(zap.Int64("collectionID", replica.collectionID), zap.Uint64("timpstamp", ts))
@ -795,15 +815,23 @@ func (replica *SegmentReplica) mergeFlushedSegments(seg *Segment, planID UniqueI
return fmt.Errorf("mismatch collection, ID=%d", seg.collectionID)
}
log.Info("merge flushed segments")
replica.segMu.Lock()
var inValidSegments []UniqueID
for _, ID := range compactedFrom {
s, ok := replica.flushedSegments[ID]
if !ok {
log.Warn("no match flushed segment to merge from", zap.Int64("segmentID", ID))
continue
if !replica.hasSegment(ID, true) {
inValidSegments = append(inValidSegments, ID)
}
}
if len(inValidSegments) > 0 {
log.Warn("no match flushed segments to merge from", zap.Int64s("invalid segmentIDs", inValidSegments))
return fmt.Errorf("invalid compactedFrom segments: %v", inValidSegments)
}
replica.segMu.Lock()
log.Info("merge flushed segments")
for _, ID := range compactedFrom {
// the existent of the segments are already checked
s := replica.flushedSegments[ID]
s.compactedTo = seg.segmentID
replica.compactedSegments[ID] = s
@ -811,12 +839,15 @@ func (replica *SegmentReplica) mergeFlushedSegments(seg *Segment, planID UniqueI
}
replica.segMu.Unlock()
seg.isNew.Store(false)
seg.isFlushed.Store(true)
// only store segments with numRows > 0
if seg.numRows > 0 {
seg.isNew.Store(false)
seg.isFlushed.Store(true)
replica.segMu.Lock()
replica.flushedSegments[seg.segmentID] = seg
replica.segMu.Unlock()
replica.segMu.Lock()
replica.flushedSegments[seg.segmentID] = seg
replica.segMu.Unlock()
}
return nil
}

View File

@ -103,6 +103,57 @@ func getSimpleFieldBinlog() *datapb.FieldBinlog {
}
}
func TestSegmentReplica_getChannelName(t *testing.T) {
var (
channelName = "TestSegmentReplica_getChannelName"
newSegments = map[UniqueID]*Segment{
100: {channelName: channelName},
101: {channelName: channelName},
102: {channelName: channelName},
}
normalSegments = map[UniqueID]*Segment{
200: {channelName: channelName},
201: {channelName: channelName},
202: {channelName: channelName},
}
flushedSegments = map[UniqueID]*Segment{
300: {channelName: channelName},
301: {channelName: channelName},
302: {channelName: channelName},
}
)
sr := &SegmentReplica{
newSegments: newSegments,
normalSegments: normalSegments,
flushedSegments: flushedSegments,
}
tests := []struct {
description string
seg UniqueID
ifExist bool
}{
{"100 exists in new segments", 100, true},
{"201 exists in normal segments", 201, true},
{"302 exists in flushed segments", 302, true},
{"400 not exists in all segments", 400, false},
}
for _, test := range tests {
t.Run(test.description, func(t *testing.T) {
chanName, err := sr.getChannelName(test.seg)
if test.ifExist {
assert.NoError(t, err)
assert.Equal(t, channelName, chanName)
} else {
assert.Error(t, err)
assert.Empty(t, chanName)
}
})
}
}
func TestSegmentReplica_getCollectionAndPartitionID(te *testing.T) {
tests := []struct {
segInNew UniqueID
@ -656,30 +707,76 @@ func TestSegmentReplica_InterfaceMethod(t *testing.T) {
primaryKeyData := &storage.Int64FieldData{
Data: []UniqueID{1},
}
sr.addFlushedSegmentWithPKs(1, 1, 0, "channel", 10, primaryKeyData)
sr.addFlushedSegmentWithPKs(2, 1, 0, "channel", 10, primaryKeyData)
require.True(t, sr.hasSegment(1, true))
require.True(t, sr.hasSegment(2, true))
tests := []struct {
description string
isValid bool
stored bool
s := &Segment{
segmentID: 3,
collectionID: 1,
partitionID: 0,
channelName: "channel",
numRows: 15,
inCompactedFrom []UniqueID
inSeg *Segment
}{
{"mismatch collection", false, false, []UniqueID{1, 2}, &Segment{
segmentID: 3,
collectionID: -1,
}},
{"no match flushed segment", false, false, []UniqueID{1, 6}, &Segment{
segmentID: 3,
collectionID: 1,
}},
{"numRows==0", true, false, []UniqueID{1, 2}, &Segment{
segmentID: 3,
collectionID: 1,
numRows: 0,
}},
{"numRows>0", true, true, []UniqueID{1, 2}, &Segment{
segmentID: 3,
collectionID: 1,
numRows: 15,
}},
}
sr.mergeFlushedSegments(s, 100, []UniqueID{1, 2})
assert.True(t, sr.hasSegment(3, true))
assert.False(t, sr.hasSegment(1, true))
assert.False(t, sr.hasSegment(2, true))
to2from := sr.listCompactedSegmentIDs()
assert.NotEmpty(t, to2from)
for _, test := range tests {
t.Run(test.description, func(t *testing.T) {
// prepare segment replica
if !sr.hasSegment(1, true) {
sr.addFlushedSegmentWithPKs(1, 1, 0, "channel", 10, primaryKeyData)
}
from, ok := to2from[3]
assert.True(t, ok)
assert.ElementsMatch(t, []UniqueID{1, 2}, from)
if !sr.hasSegment(2, true) {
sr.addFlushedSegmentWithPKs(2, 1, 0, "channel", 10, primaryKeyData)
}
if sr.hasSegment(3, true) {
sr.removeSegments(3)
}
require.True(t, sr.hasSegment(1, true))
require.True(t, sr.hasSegment(2, true))
require.False(t, sr.hasSegment(3, true))
// tests start
err := sr.mergeFlushedSegments(test.inSeg, 100, test.inCompactedFrom)
if test.isValid {
assert.NoError(t, err)
} else {
assert.Error(t, err)
}
if test.stored {
assert.True(t, sr.hasSegment(3, true))
to2from := sr.listCompactedSegmentIDs()
assert.NotEmpty(t, to2from)
from, ok := to2from[3]
assert.True(t, ok)
assert.ElementsMatch(t, []UniqueID{1, 2}, from)
} else {
assert.False(t, sr.hasSegment(3, true))
}
})
}
})
}

View File

@ -258,3 +258,17 @@ func (c *Client) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest)
}
return ret.(*commonpb.Status), err
}
// SyncSegments is the DataNode client side code for SyncSegments call.
func (c *Client) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.(datapb.DataNodeClient).SyncSegments(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}

View File

@ -381,3 +381,7 @@ func (s *Server) ResendSegmentStats(ctx context.Context, request *datapb.ResendS
func (s *Server) AddSegment(ctx context.Context, request *datapb.AddSegmentRequest) (*commonpb.Status, error) {
return s.datanode.AddSegment(ctx, request)
}
func (s *Server) SyncSegments(ctx context.Context, request *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
return s.datanode.SyncSegments(ctx, request)
}

View File

@ -134,6 +134,10 @@ func (m *MockDataNode) AddSegment(ctx context.Context, req *datapb.AddSegmentReq
return m.status, m.err
}
func (m *MockDataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
return m.status, m.err
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type mockDataCoord struct {
types.DataCoord

View File

@ -641,6 +641,53 @@ func (_c *DataNode_Stop_Call) Return(_a0 error) *DataNode_Stop_Call {
return _c
}
// SyncSegments provides a mock function with given fields: ctx, req
func (_m *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
var r0 *commonpb.Status
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SyncSegmentsRequest) *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.SyncSegmentsRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// DataNode_SyncSegments_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncSegments'
type DataNode_SyncSegments_Call struct {
*mock.Call
}
// SyncSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SyncSegmentsRequest
func (_e *DataNode_Expecter) SyncSegments(ctx interface{}, req interface{}) *DataNode_SyncSegments_Call {
return &DataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", ctx, req)}
}
func (_c *DataNode_SyncSegments_Call) Run(run func(ctx context.Context, req *datapb.SyncSegmentsRequest)) *DataNode_SyncSegments_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.SyncSegmentsRequest))
})
return _c
}
func (_c *DataNode_SyncSegments_Call) Return(_a0 *commonpb.Status, _a1 error) *DataNode_SyncSegments_Call {
_c.Call.Return(_a0, _a1)
return _c
}
// WatchDmChannels provides a mock function with given fields: ctx, req
func (_m *DataNode) WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)

View File

@ -69,6 +69,7 @@ service DataNode {
rpc Compaction(CompactionPlan) returns (common.Status) {}
rpc GetCompactionState(CompactionStateRequest) returns (CompactionStateResponse) {}
rpc SyncSegments(SyncSegmentsRequest) returns (common.Status) {}
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
rpc Import(ImportTaskRequest) returns(common.Status) {}
@ -296,7 +297,7 @@ message DeltaLogInfo {
uint64 timestamp_from = 2;
uint64 timestamp_to = 3;
string delta_log_path = 4;
int64 delta_log_size = 5;
int64 delta_log_size = 5;
}
message DataNodeTtMsg {
@ -404,6 +405,13 @@ message CompactionStateRequest {
common.MsgBase base = 1;
}
message SyncSegmentsRequest {
int64 planID = 1;
int64 compacted_to = 2;
int64 num_of_rows = 3;
repeated int64 compacted_from = 4;
}
message CompactionSegmentBinlogs {
int64 segmentID = 1;
repeated FieldBinlog fieldBinlogs = 2;
@ -469,8 +477,8 @@ message SetSegmentStateResponse {
}
message DropVirtualChannelRequest {
common.MsgBase base = 1;
string channel_name = 2;
common.MsgBase base = 1;
string channel_name = 2;
repeated DropVirtualChannelSegment segments = 3;
}

View File

@ -2854,6 +2854,69 @@ func (m *CompactionStateRequest) GetBase() *commonpb.MsgBase {
return nil
}
type SyncSegmentsRequest struct {
PlanID int64 `protobuf:"varint,1,opt,name=planID,proto3" json:"planID,omitempty"`
CompactedTo int64 `protobuf:"varint,2,opt,name=compacted_to,json=compactedTo,proto3" json:"compacted_to,omitempty"`
NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"`
CompactedFrom []int64 `protobuf:"varint,4,rep,packed,name=compacted_from,json=compactedFrom,proto3" json:"compacted_from,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *SyncSegmentsRequest) Reset() { *m = SyncSegmentsRequest{} }
func (m *SyncSegmentsRequest) String() string { return proto.CompactTextString(m) }
func (*SyncSegmentsRequest) ProtoMessage() {}
func (*SyncSegmentsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{45}
}
func (m *SyncSegmentsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SyncSegmentsRequest.Unmarshal(m, b)
}
func (m *SyncSegmentsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_SyncSegmentsRequest.Marshal(b, m, deterministic)
}
func (m *SyncSegmentsRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_SyncSegmentsRequest.Merge(m, src)
}
func (m *SyncSegmentsRequest) XXX_Size() int {
return xxx_messageInfo_SyncSegmentsRequest.Size(m)
}
func (m *SyncSegmentsRequest) XXX_DiscardUnknown() {
xxx_messageInfo_SyncSegmentsRequest.DiscardUnknown(m)
}
var xxx_messageInfo_SyncSegmentsRequest proto.InternalMessageInfo
func (m *SyncSegmentsRequest) GetPlanID() int64 {
if m != nil {
return m.PlanID
}
return 0
}
func (m *SyncSegmentsRequest) GetCompactedTo() int64 {
if m != nil {
return m.CompactedTo
}
return 0
}
func (m *SyncSegmentsRequest) GetNumOfRows() int64 {
if m != nil {
return m.NumOfRows
}
return 0
}
func (m *SyncSegmentsRequest) GetCompactedFrom() []int64 {
if m != nil {
return m.CompactedFrom
}
return nil
}
type CompactionSegmentBinlogs struct {
SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
FieldBinlogs []*FieldBinlog `protobuf:"bytes,2,rep,name=fieldBinlogs,proto3" json:"fieldBinlogs,omitempty"`
@ -2869,7 +2932,7 @@ func (m *CompactionSegmentBinlogs) Reset() { *m = CompactionSegmentBinlo
func (m *CompactionSegmentBinlogs) String() string { return proto.CompactTextString(m) }
func (*CompactionSegmentBinlogs) ProtoMessage() {}
func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{45}
return fileDescriptor_82cd95f524594f49, []int{46}
}
func (m *CompactionSegmentBinlogs) XXX_Unmarshal(b []byte) error {
@ -2942,7 +3005,7 @@ func (m *CompactionPlan) Reset() { *m = CompactionPlan{} }
func (m *CompactionPlan) String() string { return proto.CompactTextString(m) }
func (*CompactionPlan) ProtoMessage() {}
func (*CompactionPlan) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{46}
return fileDescriptor_82cd95f524594f49, []int{47}
}
func (m *CompactionPlan) XXX_Unmarshal(b []byte) error {
@ -3028,7 +3091,7 @@ func (m *CompactionResult) Reset() { *m = CompactionResult{} }
func (m *CompactionResult) String() string { return proto.CompactTextString(m) }
func (*CompactionResult) ProtoMessage() {}
func (*CompactionResult) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{47}
return fileDescriptor_82cd95f524594f49, []int{48}
}
func (m *CompactionResult) XXX_Unmarshal(b []byte) error {
@ -3104,7 +3167,7 @@ func (m *CompactionStateResult) Reset() { *m = CompactionStateResult{} }
func (m *CompactionStateResult) String() string { return proto.CompactTextString(m) }
func (*CompactionStateResult) ProtoMessage() {}
func (*CompactionStateResult) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{48}
return fileDescriptor_82cd95f524594f49, []int{49}
}
func (m *CompactionStateResult) XXX_Unmarshal(b []byte) error {
@ -3158,7 +3221,7 @@ func (m *CompactionStateResponse) Reset() { *m = CompactionStateResponse
func (m *CompactionStateResponse) String() string { return proto.CompactTextString(m) }
func (*CompactionStateResponse) ProtoMessage() {}
func (*CompactionStateResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{49}
return fileDescriptor_82cd95f524594f49, []int{50}
}
func (m *CompactionStateResponse) XXX_Unmarshal(b []byte) error {
@ -3206,7 +3269,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{}
func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) }
func (*SegmentFieldBinlogMeta) ProtoMessage() {}
func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{50}
return fileDescriptor_82cd95f524594f49, []int{51}
}
func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error {
@ -3254,7 +3317,7 @@ func (m *WatchChannelsRequest) Reset() { *m = WatchChannelsRequest{} }
func (m *WatchChannelsRequest) String() string { return proto.CompactTextString(m) }
func (*WatchChannelsRequest) ProtoMessage() {}
func (*WatchChannelsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{51}
return fileDescriptor_82cd95f524594f49, []int{52}
}
func (m *WatchChannelsRequest) XXX_Unmarshal(b []byte) error {
@ -3307,7 +3370,7 @@ func (m *WatchChannelsResponse) Reset() { *m = WatchChannelsResponse{} }
func (m *WatchChannelsResponse) String() string { return proto.CompactTextString(m) }
func (*WatchChannelsResponse) ProtoMessage() {}
func (*WatchChannelsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{52}
return fileDescriptor_82cd95f524594f49, []int{53}
}
func (m *WatchChannelsResponse) XXX_Unmarshal(b []byte) error {
@ -3348,7 +3411,7 @@ func (m *SetSegmentStateRequest) Reset() { *m = SetSegmentStateRequest{}
func (m *SetSegmentStateRequest) String() string { return proto.CompactTextString(m) }
func (*SetSegmentStateRequest) ProtoMessage() {}
func (*SetSegmentStateRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{53}
return fileDescriptor_82cd95f524594f49, []int{54}
}
func (m *SetSegmentStateRequest) XXX_Unmarshal(b []byte) error {
@ -3401,7 +3464,7 @@ func (m *SetSegmentStateResponse) Reset() { *m = SetSegmentStateResponse
func (m *SetSegmentStateResponse) String() string { return proto.CompactTextString(m) }
func (*SetSegmentStateResponse) ProtoMessage() {}
func (*SetSegmentStateResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{54}
return fileDescriptor_82cd95f524594f49, []int{55}
}
func (m *SetSegmentStateResponse) XXX_Unmarshal(b []byte) error {
@ -3442,7 +3505,7 @@ func (m *DropVirtualChannelRequest) Reset() { *m = DropVirtualChannelReq
func (m *DropVirtualChannelRequest) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelRequest) ProtoMessage() {}
func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{55}
return fileDescriptor_82cd95f524594f49, []int{56}
}
func (m *DropVirtualChannelRequest) XXX_Unmarshal(b []byte) error {
@ -3502,7 +3565,7 @@ func (m *DropVirtualChannelSegment) Reset() { *m = DropVirtualChannelSeg
func (m *DropVirtualChannelSegment) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelSegment) ProtoMessage() {}
func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{56}
return fileDescriptor_82cd95f524594f49, []int{57}
}
func (m *DropVirtualChannelSegment) XXX_Unmarshal(b []byte) error {
@ -3590,7 +3653,7 @@ func (m *DropVirtualChannelResponse) Reset() { *m = DropVirtualChannelRe
func (m *DropVirtualChannelResponse) String() string { return proto.CompactTextString(m) }
func (*DropVirtualChannelResponse) ProtoMessage() {}
func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{57}
return fileDescriptor_82cd95f524594f49, []int{58}
}
func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error {
@ -3636,7 +3699,7 @@ func (m *ImportTask) Reset() { *m = ImportTask{} }
func (m *ImportTask) String() string { return proto.CompactTextString(m) }
func (*ImportTask) ProtoMessage() {}
func (*ImportTask) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{58}
return fileDescriptor_82cd95f524594f49, []int{59}
}
func (m *ImportTask) XXX_Unmarshal(b []byte) error {
@ -3728,7 +3791,7 @@ func (m *ImportTaskState) Reset() { *m = ImportTaskState{} }
func (m *ImportTaskState) String() string { return proto.CompactTextString(m) }
func (*ImportTaskState) ProtoMessage() {}
func (*ImportTaskState) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{59}
return fileDescriptor_82cd95f524594f49, []int{60}
}
func (m *ImportTaskState) XXX_Unmarshal(b []byte) error {
@ -3807,7 +3870,7 @@ func (m *ImportTaskInfo) Reset() { *m = ImportTaskInfo{} }
func (m *ImportTaskInfo) String() string { return proto.CompactTextString(m) }
func (*ImportTaskInfo) ProtoMessage() {}
func (*ImportTaskInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{60}
return fileDescriptor_82cd95f524594f49, []int{61}
}
func (m *ImportTaskInfo) XXX_Unmarshal(b []byte) error {
@ -3932,7 +3995,7 @@ func (m *ImportTaskResponse) Reset() { *m = ImportTaskResponse{} }
func (m *ImportTaskResponse) String() string { return proto.CompactTextString(m) }
func (*ImportTaskResponse) ProtoMessage() {}
func (*ImportTaskResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{61}
return fileDescriptor_82cd95f524594f49, []int{62}
}
func (m *ImportTaskResponse) XXX_Unmarshal(b []byte) error {
@ -3980,7 +4043,7 @@ func (m *ImportTaskRequest) Reset() { *m = ImportTaskRequest{} }
func (m *ImportTaskRequest) String() string { return proto.CompactTextString(m) }
func (*ImportTaskRequest) ProtoMessage() {}
func (*ImportTaskRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{62}
return fileDescriptor_82cd95f524594f49, []int{63}
}
func (m *ImportTaskRequest) XXX_Unmarshal(b []byte) error {
@ -4034,7 +4097,7 @@ func (m *UpdateSegmentStatisticsRequest) Reset() { *m = UpdateSegmentSta
func (m *UpdateSegmentStatisticsRequest) String() string { return proto.CompactTextString(m) }
func (*UpdateSegmentStatisticsRequest) ProtoMessage() {}
func (*UpdateSegmentStatisticsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{63}
return fileDescriptor_82cd95f524594f49, []int{64}
}
func (m *UpdateSegmentStatisticsRequest) XXX_Unmarshal(b []byte) error {
@ -4080,7 +4143,7 @@ func (m *ResendSegmentStatsRequest) Reset() { *m = ResendSegmentStatsReq
func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) }
func (*ResendSegmentStatsRequest) ProtoMessage() {}
func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{64}
return fileDescriptor_82cd95f524594f49, []int{65}
}
func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error {
@ -4120,7 +4183,7 @@ func (m *ResendSegmentStatsResponse) Reset() { *m = ResendSegmentStatsRe
func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) }
func (*ResendSegmentStatsResponse) ProtoMessage() {}
func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{65}
return fileDescriptor_82cd95f524594f49, []int{66}
}
func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error {
@ -4171,7 +4234,7 @@ func (m *AddSegmentRequest) Reset() { *m = AddSegmentRequest{} }
func (m *AddSegmentRequest) String() string { return proto.CompactTextString(m) }
func (*AddSegmentRequest) ProtoMessage() {}
func (*AddSegmentRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{66}
return fileDescriptor_82cd95f524594f49, []int{67}
}
func (m *AddSegmentRequest) XXX_Unmarshal(b []byte) error {
@ -4247,7 +4310,7 @@ func (m *SegmentReferenceLock) Reset() { *m = SegmentReferenceLock{} }
func (m *SegmentReferenceLock) String() string { return proto.CompactTextString(m) }
func (*SegmentReferenceLock) ProtoMessage() {}
func (*SegmentReferenceLock) Descriptor() ([]byte, []int) {
return fileDescriptor_82cd95f524594f49, []int{67}
return fileDescriptor_82cd95f524594f49, []int{68}
}
func (m *SegmentReferenceLock) XXX_Unmarshal(b []byte) error {
@ -4337,6 +4400,7 @@ func init() {
proto.RegisterType((*SegmentFlushCompletedMsg)(nil), "milvus.proto.data.SegmentFlushCompletedMsg")
proto.RegisterType((*ChannelWatchInfo)(nil), "milvus.proto.data.ChannelWatchInfo")
proto.RegisterType((*CompactionStateRequest)(nil), "milvus.proto.data.CompactionStateRequest")
proto.RegisterType((*SyncSegmentsRequest)(nil), "milvus.proto.data.SyncSegmentsRequest")
proto.RegisterType((*CompactionSegmentBinlogs)(nil), "milvus.proto.data.CompactionSegmentBinlogs")
proto.RegisterType((*CompactionPlan)(nil), "milvus.proto.data.CompactionPlan")
proto.RegisterType((*CompactionResult)(nil), "milvus.proto.data.CompactionResult")
@ -4365,250 +4429,253 @@ func init() {
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 3878 bytes of a gzipped FileDescriptorProto
// 3935 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0x5b, 0x6f, 0x1b, 0x47,
0x77, 0x5e, 0xde, 0x44, 0x1e, 0x5e, 0x44, 0x8d, 0x1d, 0x89, 0xa6, 0xef, 0xeb, 0xd8, 0x91, 0xfd,
0x39, 0x76, 0x22, 0x37, 0xf8, 0x82, 0xfa, 0x4b, 0x02, 0xcb, 0xb2, 0x64, 0xb6, 0x92, 0x23, 0xaf,
0xe4, 0x18, 0x68, 0x0a, 0x10, 0x2b, 0xee, 0x88, 0xda, 0x88, 0xbb, 0x4b, 0xef, 0x2e, 0x2d, 0x2b,
0x7d, 0x48, 0xd0, 0x00, 0x05, 0x52, 0x14, 0x4d, 0x2f, 0x28, 0xd0, 0x3e, 0x14, 0x28, 0xfa, 0x94,
0x16, 0x28, 0x50, 0x20, 0xe8, 0x43, 0x5b, 0xe4, 0x3d, 0x68, 0x1f, 0x8a, 0x3e, 0xf6, 0x0f, 0xb4,
0xfd, 0x15, 0x45, 0x31, 0x97, 0x9d, 0xbd, 0x93, 0x2b, 0xd2, 0x8e, 0x8b, 0xbe, 0x69, 0xce, 0x9e,
0x73, 0xe6, 0xcc, 0xcc, 0xb9, 0xcf, 0x50, 0xd0, 0xd4, 0x54, 0x57, 0xed, 0xf6, 0x2c, 0xcb, 0xd6,
0x6e, 0x0f, 0x6d, 0xcb, 0xb5, 0xd0, 0x82, 0xa1, 0x0f, 0x5e, 0x8c, 0x1c, 0x36, 0xba, 0x4d, 0x3e,
0xb7, 0x6b, 0x3d, 0xcb, 0x30, 0x2c, 0x93, 0x81, 0xda, 0x0d, 0xdd, 0x74, 0xb1, 0x6d, 0xaa, 0x03,
0x3e, 0xae, 0x05, 0x09, 0xda, 0x35, 0xa7, 0x77, 0x80, 0x0d, 0x95, 0x8d, 0xe4, 0x39, 0x28, 0x3e,
0x34, 0x86, 0xee, 0xb1, 0xfc, 0xe7, 0x12, 0xd4, 0xd6, 0x07, 0x23, 0xe7, 0x40, 0xc1, 0xcf, 0x47,
0xd8, 0x71, 0xd1, 0x7b, 0x50, 0xd8, 0x53, 0x1d, 0xdc, 0x92, 0x2e, 0x4b, 0xcb, 0xd5, 0x95, 0xf3,
0xb7, 0x43, 0xb3, 0xf2, 0xf9, 0xb6, 0x9c, 0xfe, 0xaa, 0xea, 0x60, 0x85, 0x62, 0x22, 0x04, 0x05,
0x6d, 0xaf, 0xb3, 0xd6, 0xca, 0x5d, 0x96, 0x96, 0xf3, 0x0a, 0xfd, 0x1b, 0x5d, 0x04, 0x70, 0x70,
0xdf, 0xc0, 0xa6, 0xdb, 0x59, 0x73, 0x5a, 0xf9, 0xcb, 0xf9, 0xe5, 0xbc, 0x12, 0x80, 0x20, 0x19,
0x6a, 0x3d, 0x6b, 0x30, 0xc0, 0x3d, 0x57, 0xb7, 0xcc, 0xce, 0x5a, 0xab, 0x40, 0x69, 0x43, 0x30,
0xf9, 0x3f, 0x25, 0xa8, 0x73, 0xd1, 0x9c, 0xa1, 0x65, 0x3a, 0x18, 0xdd, 0x85, 0x92, 0xe3, 0xaa,
0xee, 0xc8, 0xe1, 0xd2, 0x9d, 0x4b, 0x94, 0x6e, 0x87, 0xa2, 0x28, 0x1c, 0x35, 0x51, 0xbc, 0xe8,
0xf4, 0xf9, 0xf8, 0xf4, 0x91, 0x25, 0x14, 0x62, 0x4b, 0x58, 0x86, 0xf9, 0x7d, 0x22, 0xdd, 0x8e,
0x8f, 0x54, 0xa4, 0x48, 0x51, 0x30, 0xe1, 0xe4, 0xea, 0x06, 0xfe, 0x74, 0x7f, 0x07, 0xab, 0x83,
0x56, 0x89, 0xce, 0x15, 0x80, 0xc8, 0xff, 0x2e, 0x41, 0x53, 0xa0, 0x7b, 0xe7, 0x70, 0x06, 0x8a,
0x3d, 0x6b, 0x64, 0xba, 0x74, 0xa9, 0x75, 0x85, 0x0d, 0xd0, 0x15, 0xa8, 0xf5, 0x0e, 0x54, 0xd3,
0xc4, 0x83, 0xae, 0xa9, 0x1a, 0x98, 0x2e, 0xaa, 0xa2, 0x54, 0x39, 0xec, 0xb1, 0x6a, 0xe0, 0x4c,
0x6b, 0xbb, 0x0c, 0xd5, 0xa1, 0x6a, 0xbb, 0x7a, 0x68, 0xf7, 0x83, 0x20, 0xd4, 0x86, 0xb2, 0xee,
0x74, 0x8c, 0xa1, 0x65, 0xbb, 0xad, 0xe2, 0x65, 0x69, 0xb9, 0xac, 0x88, 0x31, 0x99, 0x41, 0xa7,
0x7f, 0xed, 0xaa, 0xce, 0x61, 0x67, 0x8d, 0xaf, 0x28, 0x04, 0x93, 0xff, 0x4a, 0x82, 0xc5, 0xfb,
0x8e, 0xa3, 0xf7, 0xcd, 0xd8, 0xca, 0x16, 0xa1, 0x64, 0x5a, 0x1a, 0xee, 0xac, 0xd1, 0xa5, 0xe5,
0x15, 0x3e, 0x42, 0xe7, 0xa0, 0x32, 0xc4, 0xd8, 0xee, 0xda, 0xd6, 0xc0, 0x5b, 0x58, 0x99, 0x00,
0x14, 0x6b, 0x80, 0xd1, 0x13, 0x58, 0x70, 0x22, 0x8c, 0x98, 0x5e, 0x55, 0x57, 0xae, 0xde, 0x8e,
0x59, 0xc6, 0xed, 0xe8, 0xa4, 0x4a, 0x9c, 0x5a, 0xfe, 0x3a, 0x07, 0xa7, 0x05, 0x1e, 0x93, 0x95,
0xfc, 0x4d, 0x76, 0xde, 0xc1, 0x7d, 0x21, 0x1e, 0x1b, 0x64, 0xd9, 0x79, 0x71, 0x64, 0xf9, 0xe0,
0x91, 0x65, 0x50, 0xf5, 0xe8, 0x79, 0x14, 0xe3, 0xe7, 0x71, 0x09, 0xaa, 0xf8, 0xe5, 0x50, 0xb7,
0x71, 0x97, 0x28, 0x0e, 0xdd, 0xf2, 0x82, 0x02, 0x0c, 0xb4, 0xab, 0x1b, 0x41, 0xdb, 0x98, 0xcb,
0x6c, 0x1b, 0xf2, 0x5f, 0x4b, 0xb0, 0x14, 0x3b, 0x25, 0x6e, 0x6c, 0x0a, 0x34, 0xe9, 0xca, 0xfd,
0x9d, 0x21, 0x66, 0x47, 0x36, 0xfc, 0xfa, 0xb8, 0x0d, 0xf7, 0xd1, 0x95, 0x18, 0x7d, 0x40, 0xc8,
0x5c, 0x76, 0x21, 0x0f, 0x61, 0x69, 0x03, 0xbb, 0x7c, 0x02, 0xf2, 0x0d, 0x3b, 0xd3, 0x3b, 0xab,
0xb0, 0x55, 0xe7, 0xa2, 0x56, 0x2d, 0xff, 0x7d, 0x4e, 0xd8, 0x22, 0x9d, 0xaa, 0x63, 0xee, 0x5b,
0xe8, 0x3c, 0x54, 0x04, 0x0a, 0xd7, 0x0a, 0x1f, 0x80, 0x7e, 0x09, 0x45, 0x22, 0x29, 0x53, 0x89,
0xc6, 0xca, 0x95, 0xe4, 0x35, 0x05, 0x78, 0x2a, 0x0c, 0x1f, 0x75, 0xa0, 0xe1, 0xb8, 0xaa, 0xed,
0x76, 0x87, 0x96, 0x43, 0xcf, 0x99, 0x2a, 0x4e, 0x75, 0x45, 0x0e, 0x73, 0x10, 0x6e, 0x7d, 0xcb,
0xe9, 0x6f, 0x73, 0x4c, 0xa5, 0x4e, 0x29, 0xbd, 0x21, 0x7a, 0x08, 0x35, 0x6c, 0x6a, 0x3e, 0xa3,
0x42, 0x66, 0x46, 0x55, 0x6c, 0x6a, 0x82, 0x8d, 0x7f, 0x3e, 0xc5, 0xec, 0xe7, 0xf3, 0x07, 0x12,
0xb4, 0xe2, 0x07, 0x34, 0x8b, 0xcb, 0xbe, 0xc7, 0x88, 0x30, 0x3b, 0xa0, 0xb1, 0x16, 0x2e, 0x0e,
0x49, 0xe1, 0x24, 0xf2, 0x9f, 0x49, 0xf0, 0x96, 0x2f, 0x0e, 0xfd, 0xf4, 0xba, 0xb4, 0x05, 0xdd,
0x84, 0xa6, 0x6e, 0xf6, 0x06, 0x23, 0x0d, 0x3f, 0x35, 0x1f, 0x61, 0x75, 0xe0, 0x1e, 0x1c, 0xd3,
0x33, 0x2c, 0x2b, 0x31, 0xb8, 0xfc, 0x8d, 0x04, 0x8b, 0x51, 0xb9, 0x66, 0xd9, 0xa4, 0x5f, 0x83,
0xa2, 0x6e, 0xee, 0x5b, 0xde, 0x1e, 0x5d, 0x1c, 0x63, 0x94, 0x64, 0x2e, 0x86, 0x2c, 0x1b, 0x70,
0x6e, 0x03, 0xbb, 0x1d, 0xd3, 0xc1, 0xb6, 0xbb, 0xaa, 0x9b, 0x03, 0xab, 0xbf, 0xad, 0xba, 0x07,
0x33, 0x18, 0x54, 0xc8, 0x36, 0x72, 0x11, 0xdb, 0x90, 0xbf, 0x97, 0xe0, 0x7c, 0xf2, 0x7c, 0x7c,
0xe9, 0x6d, 0x28, 0xef, 0xeb, 0x78, 0xa0, 0x91, 0xfd, 0x95, 0xe8, 0xfe, 0x8a, 0x31, 0x31, 0xac,
0x21, 0x41, 0xe6, 0x2b, 0xbc, 0x92, 0xa2, 0xcd, 0x3b, 0xae, 0xad, 0x9b, 0xfd, 0x4d, 0xdd, 0x71,
0x15, 0x86, 0x1f, 0xd8, 0xcf, 0x7c, 0x76, 0x35, 0xfe, 0x7d, 0x09, 0x2e, 0x6e, 0x60, 0xf7, 0x81,
0xf0, 0xcb, 0xe4, 0xbb, 0xee, 0xb8, 0x7a, 0xcf, 0x79, 0xb5, 0xb9, 0x51, 0x86, 0x00, 0x2d, 0x7f,
0x27, 0xc1, 0xa5, 0x54, 0x61, 0xf8, 0xd6, 0x71, 0xbf, 0xe3, 0x79, 0xe5, 0x64, 0xbf, 0xf3, 0x9b,
0xf8, 0xf8, 0x33, 0x75, 0x30, 0xc2, 0xdb, 0xaa, 0x6e, 0x33, 0xbf, 0x33, 0xa5, 0x17, 0xfe, 0x3b,
0x09, 0x2e, 0x6c, 0x60, 0x77, 0xdb, 0x8b, 0x49, 0x6f, 0x70, 0x77, 0x08, 0x4e, 0x20, 0x36, 0x7a,
0xc9, 0x59, 0x08, 0x26, 0xff, 0x21, 0x3b, 0xce, 0x44, 0x79, 0xdf, 0xc8, 0x06, 0x5e, 0xa4, 0x96,
0x10, 0x30, 0xc9, 0x07, 0x2c, 0x75, 0xe0, 0xdb, 0x27, 0xff, 0xa5, 0x04, 0x67, 0xef, 0xf7, 0x9e,
0x8f, 0x74, 0x1b, 0x73, 0xa4, 0x4d, 0xab, 0x77, 0x38, 0xfd, 0xe6, 0xfa, 0x69, 0x56, 0x2e, 0x94,
0x66, 0x4d, 0x4a, 0xcd, 0x17, 0xa1, 0xe4, 0xb2, 0xbc, 0x8e, 0x65, 0x2a, 0x7c, 0x44, 0xe5, 0x53,
0xf0, 0x00, 0xab, 0xce, 0xff, 0x4d, 0xf9, 0xbe, 0x2b, 0x40, 0xed, 0x33, 0x9e, 0x8e, 0xd1, 0xa8,
0x1d, 0xd5, 0x24, 0x29, 0x39, 0xf1, 0x0a, 0x64, 0x70, 0x49, 0x49, 0xdd, 0x06, 0xd4, 0x1d, 0x8c,
0x0f, 0xa7, 0x89, 0xd1, 0x35, 0x42, 0x28, 0x62, 0xeb, 0x26, 0x2c, 0x8c, 0x4c, 0x5a, 0x1a, 0x60,
0x8d, 0x6f, 0x20, 0xd3, 0xdc, 0xc9, 0xbe, 0x3b, 0x4e, 0x88, 0x1e, 0xf1, 0xea, 0x23, 0xc0, 0xab,
0x98, 0x89, 0x57, 0x94, 0x0c, 0x75, 0xa0, 0xa9, 0xd9, 0xd6, 0x70, 0x88, 0xb5, 0xae, 0xe3, 0xb1,
0x2a, 0x65, 0x63, 0xc5, 0xe9, 0x04, 0xab, 0xf7, 0xe0, 0x74, 0x54, 0xd2, 0x8e, 0x46, 0x12, 0x52,
0x72, 0x86, 0x49, 0x9f, 0xd0, 0x2d, 0x58, 0x88, 0xe3, 0x97, 0x29, 0x7e, 0xfc, 0x03, 0x7a, 0x17,
0x50, 0x44, 0x54, 0x82, 0x5e, 0x61, 0xe8, 0x61, 0x61, 0x3a, 0x9a, 0x23, 0x7f, 0x2b, 0xc1, 0xe2,
0x33, 0xd5, 0xed, 0x1d, 0xac, 0x19, 0xdc, 0xd6, 0x66, 0xf0, 0x55, 0x1f, 0x41, 0xe5, 0x05, 0xd7,
0x0b, 0x2f, 0x20, 0x5d, 0x4a, 0xd8, 0x9f, 0xa0, 0x06, 0x2a, 0x3e, 0x85, 0xfc, 0x93, 0x04, 0x67,
0xd6, 0x03, 0x75, 0xe1, 0x1b, 0xf0, 0x9a, 0x93, 0x0a, 0xda, 0xeb, 0xd0, 0x30, 0x54, 0xfb, 0x30,
0x56, 0xcf, 0x46, 0xa0, 0xf2, 0x4b, 0x00, 0x3e, 0xda, 0x72, 0xfa, 0x53, 0xc8, 0xff, 0x21, 0xcc,
0xf1, 0x59, 0xb9, 0xfb, 0x9c, 0xa4, 0x67, 0x1e, 0xba, 0xfc, 0x2f, 0x12, 0x34, 0xfc, 0x90, 0x48,
0x8d, 0xbc, 0x01, 0x39, 0x61, 0xda, 0xb9, 0xce, 0x1a, 0xfa, 0x08, 0x4a, 0xac, 0xd1, 0xc1, 0x79,
0x5f, 0x0b, 0xf3, 0xe6, 0x4d, 0x90, 0x40, 0x5c, 0xa5, 0x00, 0x85, 0x13, 0x91, 0x3d, 0x12, 0x51,
0x44, 0x38, 0x1f, 0x1f, 0x82, 0x3a, 0x30, 0x1f, 0x4e, 0xd9, 0x3d, 0x13, 0xbe, 0x9c, 0x16, 0x3c,
0xd6, 0x54, 0x57, 0xa5, 0xb1, 0xa3, 0x11, 0xca, 0xd8, 0x1d, 0xf9, 0x8f, 0x4b, 0x50, 0x0d, 0xac,
0x32, 0xb6, 0x92, 0xe8, 0x91, 0xe6, 0x26, 0xd7, 0x8d, 0xf9, 0x78, 0xdd, 0x78, 0x0d, 0x1a, 0x3a,
0x4d, 0xbe, 0xba, 0x5c, 0x15, 0xa9, 0xd7, 0xac, 0x28, 0x75, 0x06, 0xe5, 0x76, 0x81, 0x2e, 0x42,
0xd5, 0x1c, 0x19, 0x5d, 0x6b, 0xbf, 0x6b, 0x5b, 0x47, 0x0e, 0x2f, 0x40, 0x2b, 0xe6, 0xc8, 0xf8,
0x74, 0x5f, 0xb1, 0x8e, 0x1c, 0xbf, 0xc6, 0x29, 0x9d, 0xb0, 0xc6, 0xb9, 0x08, 0x55, 0x43, 0x7d,
0x49, 0xb8, 0x76, 0xcd, 0x91, 0x41, 0x6b, 0xd3, 0xbc, 0x52, 0x31, 0xd4, 0x97, 0x8a, 0x75, 0xf4,
0x78, 0x64, 0xa0, 0x65, 0x68, 0x0e, 0x54, 0xc7, 0xed, 0x06, 0x8b, 0xdb, 0x32, 0x2d, 0x6e, 0x1b,
0x04, 0xfe, 0xd0, 0x2f, 0x70, 0xe3, 0xd5, 0x52, 0x65, 0x86, 0x6a, 0x49, 0x33, 0x06, 0x3e, 0x23,
0xc8, 0x5e, 0x2d, 0x69, 0xc6, 0x40, 0xb0, 0xf9, 0x10, 0xe6, 0xf6, 0x68, 0x4a, 0xeb, 0xb4, 0xaa,
0xa9, 0x0e, 0x73, 0x9d, 0x64, 0xb3, 0x2c, 0xf3, 0x55, 0x3c, 0x74, 0xf4, 0x2b, 0xa8, 0xd0, 0x4c,
0x82, 0xd2, 0xd6, 0x32, 0xd1, 0xfa, 0x04, 0x84, 0x5a, 0xc3, 0x03, 0x57, 0xa5, 0xd4, 0xf5, 0x6c,
0xd4, 0x82, 0x80, 0x38, 0xe9, 0x9e, 0x8d, 0x55, 0x17, 0x6b, 0xab, 0xc7, 0x0f, 0x2c, 0x63, 0xa8,
0x52, 0x65, 0x6a, 0x35, 0x68, 0xd9, 0x92, 0xf4, 0x89, 0x38, 0x86, 0x9e, 0x18, 0xad, 0xdb, 0x96,
0xd1, 0x9a, 0x67, 0x8e, 0x21, 0x0c, 0x45, 0x17, 0x00, 0x3c, 0xf7, 0xac, 0xba, 0xad, 0x26, 0x3d,
0xc5, 0x0a, 0x87, 0xdc, 0xa7, 0xbd, 0x2b, 0xdd, 0xe9, 0xb2, 0x2e, 0x91, 0x6e, 0xf6, 0x5b, 0x0b,
0x74, 0xc6, 0xaa, 0xd7, 0x56, 0xd2, 0xcd, 0xbe, 0xfc, 0x15, 0x9c, 0xf1, 0x95, 0x28, 0x70, 0x60,
0xf1, 0xb3, 0x97, 0xa6, 0x3d, 0xfb, 0xf1, 0xf5, 0xca, 0xbf, 0x15, 0x60, 0x71, 0x47, 0x7d, 0x81,
0x5f, 0x7f, 0x69, 0x94, 0xc9, 0x65, 0x6f, 0xc2, 0x02, 0xad, 0x86, 0x56, 0x02, 0xf2, 0x8c, 0xc9,
0x19, 0x82, 0x27, 0x1e, 0x27, 0x44, 0x9f, 0x90, 0x64, 0x07, 0xf7, 0x0e, 0xb7, 0x2d, 0xdd, 0xcf,
0x17, 0x2e, 0x24, 0xf0, 0x79, 0x20, 0xb0, 0x94, 0x20, 0x05, 0xda, 0x8e, 0x7b, 0x3f, 0x96, 0x29,
0xbc, 0x33, 0xb6, 0x40, 0xf7, 0x77, 0x3f, 0xea, 0x04, 0x51, 0x0b, 0xe6, 0x78, 0x98, 0xa7, 0xae,
0xa1, 0xac, 0x78, 0x43, 0xb4, 0x0d, 0xa7, 0xd9, 0x0a, 0x76, 0xb8, 0xde, 0xb3, 0xc5, 0x97, 0x33,
0x2d, 0x3e, 0x89, 0x34, 0x6c, 0x36, 0x95, 0x93, 0x9a, 0x4d, 0x0b, 0xe6, 0xb8, 0x2a, 0x53, 0x77,
0x51, 0x56, 0xbc, 0x21, 0x39, 0x66, 0x5f, 0xa9, 0xab, 0xf4, 0x9b, 0x0f, 0x20, 0x65, 0x25, 0xf8,
0xfb, 0x39, 0xa1, 0x95, 0xf4, 0x31, 0x94, 0x85, 0x86, 0xe7, 0x32, 0x6b, 0xb8, 0xa0, 0x89, 0xba,
0xf1, 0x7c, 0xc4, 0x8d, 0xcb, 0xff, 0x2a, 0x41, 0x6d, 0x8d, 0x2c, 0x69, 0xd3, 0xea, 0xd3, 0xa0,
0x73, 0x0d, 0x1a, 0x36, 0xee, 0x59, 0xb6, 0xd6, 0xc5, 0xa6, 0x6b, 0xeb, 0x98, 0x75, 0x20, 0x0a,
0x4a, 0x9d, 0x41, 0x1f, 0x32, 0x20, 0x41, 0x23, 0x9e, 0xd9, 0x71, 0x55, 0x63, 0xd8, 0xdd, 0x27,
0x1e, 0x20, 0xc7, 0xd0, 0x04, 0x94, 0x3a, 0x80, 0x2b, 0x50, 0xf3, 0xd1, 0x5c, 0x8b, 0xce, 0x5f,
0x50, 0xaa, 0x02, 0xb6, 0x6b, 0xa1, 0xb7, 0xa1, 0x41, 0xf7, 0xb4, 0x3b, 0xb0, 0xfa, 0x5d, 0x52,
0xad, 0xf3, 0x78, 0x54, 0xd3, 0xb8, 0x58, 0xe4, 0xac, 0xc2, 0x58, 0x8e, 0xfe, 0x25, 0xe6, 0x11,
0x49, 0x60, 0xed, 0xe8, 0x5f, 0x62, 0x92, 0x0e, 0xd4, 0x49, 0x78, 0x7d, 0x6c, 0x69, 0x78, 0x77,
0xca, 0x64, 0x24, 0x43, 0x5b, 0xf7, 0x3c, 0x54, 0xc4, 0x0a, 0xf8, 0x92, 0x7c, 0x00, 0x5a, 0x87,
0x86, 0x97, 0x36, 0x77, 0x59, 0x35, 0x59, 0x48, 0x4d, 0x0e, 0x03, 0x01, 0xd2, 0x51, 0xea, 0x1e,
0x19, 0x1d, 0xca, 0xeb, 0x50, 0x0b, 0x7e, 0x26, 0xb3, 0xee, 0x44, 0x15, 0x45, 0x00, 0x88, 0x36,
0x3e, 0x1e, 0x19, 0xe4, 0x4c, 0xb9, 0x63, 0xf1, 0x86, 0xf2, 0x37, 0x12, 0xd4, 0x79, 0x54, 0xdf,
0x11, 0x17, 0x20, 0x74, 0x69, 0x12, 0x5d, 0x1a, 0xfd, 0x1b, 0xfd, 0x7a, 0xb8, 0x67, 0xf9, 0x76,
0xa2, 0x13, 0xa0, 0x4c, 0x68, 0x02, 0x1d, 0x0a, 0xe9, 0x59, 0xfa, 0x17, 0x5f, 0x13, 0x45, 0xe3,
0x47, 0x43, 0x15, 0xad, 0x05, 0x73, 0xaa, 0xa6, 0xd9, 0xd8, 0x71, 0xb8, 0x1c, 0xde, 0x90, 0x7c,
0x79, 0x81, 0x6d, 0xc7, 0x53, 0xf9, 0xbc, 0xe2, 0x0d, 0xd1, 0xaf, 0xa0, 0x2c, 0x32, 0xee, 0x7c,
0x52, 0x96, 0x15, 0x94, 0x93, 0x57, 0xdb, 0x82, 0x42, 0xfe, 0x87, 0x1c, 0x34, 0xf8, 0x86, 0xad,
0xf2, 0xb0, 0x3b, 0xde, 0xf8, 0x56, 0xa1, 0xb6, 0xef, 0xdb, 0xfe, 0xb8, 0xbe, 0x5a, 0xd0, 0x45,
0x84, 0x68, 0x26, 0x19, 0x60, 0x38, 0xf0, 0x17, 0x66, 0x0a, 0xfc, 0xc5, 0x93, 0x7a, 0xb0, 0x78,
0x2a, 0x58, 0x4a, 0x48, 0x05, 0xe5, 0xdf, 0x86, 0x6a, 0x80, 0x01, 0xf5, 0xd0, 0xac, 0x21, 0xc7,
0x77, 0xcc, 0x1b, 0xa2, 0xbb, 0x7e, 0xfa, 0xc3, 0xb6, 0xea, 0x6c, 0x82, 0x2c, 0x91, 0xcc, 0x47,
0xfe, 0x1b, 0x09, 0x4a, 0x9c, 0xf3, 0x25, 0xa8, 0x72, 0xa7, 0x43, 0x53, 0x43, 0xc6, 0x1d, 0x38,
0x88, 0xe4, 0x86, 0xaf, 0xce, 0xeb, 0x9c, 0x85, 0x72, 0xc4, 0xdf, 0xcc, 0xf1, 0xb0, 0xe0, 0x7d,
0x0a, 0x38, 0x19, 0xf2, 0x89, 0xfa, 0x97, 0x9f, 0x24, 0x7a, 0xf3, 0xa0, 0xe0, 0x9e, 0xf5, 0x02,
0xdb, 0xc7, 0xb3, 0xb7, 0x6c, 0xef, 0x05, 0x14, 0x3a, 0x63, 0x09, 0x29, 0x08, 0xd0, 0x3d, 0x7f,
0xbb, 0xf3, 0x49, 0xfd, 0xaa, 0xa0, 0x87, 0xe1, 0xea, 0xe8, 0x6f, 0xfb, 0x1f, 0xb1, 0xe6, 0x73,
0x78, 0x29, 0xd3, 0xe6, 0x35, 0xaf, 0xa4, 0x32, 0x91, 0xff, 0x54, 0x82, 0xb3, 0x1b, 0xd8, 0x5d,
0x0f, 0xb7, 0x23, 0xde, 0xb4, 0x54, 0x06, 0xb4, 0x93, 0x84, 0x9a, 0xe5, 0xd4, 0xdb, 0x50, 0x16,
0x8d, 0x15, 0x76, 0x85, 0x20, 0xc6, 0xf2, 0xef, 0x49, 0xd0, 0xe2, 0xb3, 0xd0, 0x39, 0x49, 0xd6,
0x3d, 0xc0, 0x2e, 0xd6, 0x7e, 0xee, 0xd2, 0xfa, 0x47, 0x09, 0x9a, 0x41, 0x8f, 0x4f, 0x9d, 0xf6,
0x07, 0x50, 0xa4, 0x1d, 0x0c, 0x2e, 0xc1, 0x44, 0x65, 0x65, 0xd8, 0xc4, 0x65, 0xd0, 0x34, 0x6f,
0x57, 0x04, 0x27, 0x3e, 0xf4, 0xc3, 0x4e, 0xfe, 0xe4, 0x61, 0x87, 0x87, 0x61, 0x6b, 0x44, 0xf8,
0xb2, 0xd6, 0x9f, 0x0f, 0x90, 0x7f, 0x03, 0x16, 0xfd, 0x8a, 0x85, 0xd1, 0x4d, 0xab, 0x49, 0xf2,
0x0f, 0x39, 0x68, 0x05, 0x98, 0xfd, 0xdc, 0x31, 0x24, 0x25, 0xf3, 0xcd, 0xbf, 0xa2, 0xcc, 0xb7,
0x30, 0x7b, 0xdc, 0x28, 0x26, 0xc5, 0x8d, 0x7f, 0xce, 0x41, 0xc3, 0xdf, 0xb5, 0xed, 0x81, 0x6a,
0xa2, 0x45, 0x28, 0x0d, 0x07, 0xaa, 0xdf, 0x7b, 0xe5, 0x23, 0xb4, 0x23, 0x72, 0xa6, 0xf0, 0x3e,
0xfd, 0x22, 0x49, 0x1f, 0x52, 0x0e, 0x42, 0x89, 0xb0, 0x20, 0xd5, 0x27, 0x2b, 0x4e, 0x68, 0x0f,
0x81, 0xe7, 0x69, 0x4c, 0xf1, 0x74, 0x03, 0xa3, 0x5b, 0x80, 0xb8, 0xb6, 0x74, 0x75, 0xb3, 0xeb,
0xe0, 0x9e, 0x65, 0x6a, 0x4c, 0x8f, 0x8a, 0x4a, 0x93, 0x7f, 0xe9, 0x98, 0x3b, 0x0c, 0x8e, 0x3e,
0x80, 0x82, 0x7b, 0x3c, 0x64, 0x11, 0xa1, 0x91, 0xe8, 0x69, 0x7d, 0xb9, 0x76, 0x8f, 0x87, 0x58,
0xa1, 0xe8, 0xde, 0x4b, 0x0f, 0xd7, 0x56, 0x5f, 0xf0, 0xf0, 0x5a, 0x50, 0x02, 0x10, 0x62, 0x19,
0xde, 0x1e, 0xce, 0xb1, 0x30, 0xc4, 0x87, 0xf2, 0x3f, 0xe6, 0xa0, 0xe9, 0xb3, 0x54, 0xb0, 0x33,
0x1a, 0xb8, 0xa9, 0xfb, 0x37, 0xbe, 0xb0, 0x9c, 0x94, 0x83, 0x7c, 0x02, 0x55, 0x7e, 0x9e, 0x27,
0xd0, 0x07, 0x60, 0x24, 0x9b, 0x63, 0x14, 0xb4, 0xf8, 0x8a, 0x14, 0xb4, 0x74, 0x42, 0x05, 0x95,
0xbf, 0x97, 0xe0, 0xad, 0x98, 0xf1, 0x8f, 0xdd, 0xc0, 0xf1, 0xe9, 0x2f, 0x77, 0x0a, 0x51, 0x96,
0xdc, 0x0f, 0xdd, 0x83, 0x92, 0x4d, 0xb9, 0xf3, 0x9b, 0x80, 0xab, 0x63, 0x95, 0x83, 0x09, 0xa2,
0x70, 0x12, 0xf9, 0x4f, 0x24, 0x58, 0x8a, 0x8b, 0x3a, 0x43, 0x70, 0x59, 0x85, 0x39, 0xc6, 0xda,
0xb3, 0xa1, 0xe5, 0xf1, 0x36, 0xe4, 0x6f, 0x8e, 0xe2, 0x11, 0xca, 0x3b, 0xb0, 0xe8, 0xc5, 0x20,
0x7f, 0x83, 0xb7, 0xb0, 0xab, 0x8e, 0x49, 0xfe, 0x2e, 0x41, 0x95, 0xe5, 0x16, 0x2c, 0xa9, 0x62,
0x65, 0x13, 0xec, 0x89, 0x6e, 0x03, 0x49, 0xf4, 0xce, 0x50, 0x27, 0x1e, 0x6d, 0xbd, 0x67, 0xb9,
0x96, 0x91, 0x45, 0x55, 0x46, 0x2a, 0x30, 0xb6, 0xb4, 0x8a, 0x12, 0x82, 0x25, 0xb5, 0x62, 0xf3,
0x53, 0xb6, 0x62, 0x37, 0xe1, 0xad, 0x88, 0xa8, 0x33, 0x1c, 0x09, 0x59, 0xf9, 0xe2, 0x4e, 0xf8,
0x3d, 0xc4, 0xf4, 0x59, 0xcd, 0x05, 0xd1, 0xb4, 0xef, 0xea, 0x5a, 0xd4, 0xd6, 0x35, 0xf4, 0x31,
0x54, 0x4c, 0x7c, 0xd4, 0x0d, 0x06, 0xd5, 0x0c, 0xbd, 0xd9, 0xb2, 0x89, 0x8f, 0xe8, 0x5f, 0xf2,
0x63, 0x58, 0x8a, 0x89, 0x3a, 0xcb, 0xda, 0xff, 0x49, 0x82, 0xb3, 0x6b, 0xb6, 0x35, 0xfc, 0x4c,
0xb7, 0xdd, 0x91, 0x3a, 0x08, 0x5f, 0x71, 0xbe, 0x9e, 0xf2, 0xfc, 0x51, 0x20, 0xbd, 0x62, 0x0a,
0x70, 0x2b, 0xc1, 0x04, 0xe2, 0x42, 0xf1, 0x45, 0x07, 0x92, 0xb1, 0xff, 0xca, 0x27, 0x09, 0xcf,
0xf1, 0x26, 0x04, 0xfe, 0x2c, 0xd9, 0x67, 0x62, 0x37, 0x2f, 0x3f, 0x6d, 0x37, 0x2f, 0xc5, 0x0b,
0x17, 0x5e, 0x91, 0x17, 0x3e, 0x71, 0x79, 0xf9, 0x08, 0xc2, 0x9d, 0x56, 0x1a, 0xfe, 0xa6, 0x6a,
0xd1, 0xae, 0x02, 0xf8, 0x5d, 0x47, 0xfe, 0x9c, 0x2d, 0x0b, 0x9b, 0x00, 0x15, 0x39, 0x2d, 0x11,
0xf1, 0xe8, 0x85, 0x42, 0xa8, 0x0f, 0xf6, 0x04, 0xda, 0x49, 0x5a, 0x3a, 0x8b, 0xe6, 0xff, 0x90,
0x03, 0xe8, 0x88, 0x17, 0x90, 0xd3, 0x39, 0xf3, 0xab, 0x50, 0xf7, 0x15, 0xc6, 0xb7, 0xf7, 0xa0,
0x16, 0x69, 0xc4, 0x24, 0x44, 0xc1, 0x42, 0x70, 0x62, 0x45, 0x8c, 0x46, 0xf9, 0x04, 0xac, 0x86,
0x29, 0x45, 0xd4, 0x7f, 0x9e, 0x83, 0x8a, 0x6d, 0x1d, 0x75, 0x89, 0x99, 0x69, 0xde, 0x13, 0x4f,
0xdb, 0x3a, 0x22, 0xc6, 0xa7, 0xa1, 0x25, 0x98, 0x73, 0x55, 0xe7, 0x90, 0xf0, 0x2f, 0x05, 0x6e,
0xd9, 0x35, 0x74, 0x06, 0x8a, 0xfb, 0xfa, 0x00, 0xb3, 0x4b, 0xdd, 0x8a, 0xc2, 0x06, 0xe8, 0x97,
0xde, 0x5b, 0xa4, 0x72, 0xe6, 0x97, 0x14, 0xec, 0x39, 0xd2, 0x4f, 0x12, 0xcc, 0xfb, 0xbb, 0x46,
0x1d, 0x10, 0xf1, 0x69, 0xd4, 0x9f, 0x3d, 0xb0, 0x34, 0xe6, 0x2a, 0x1a, 0x29, 0x2e, 0x9d, 0x11,
0x32, 0xaf, 0xe5, 0x93, 0x8c, 0xab, 0xb7, 0xc8, 0xba, 0xc8, 0xa2, 0x75, 0xcd, 0xbb, 0xdc, 0x2b,
0xd9, 0xd6, 0x51, 0x47, 0x13, 0xbb, 0xc1, 0xde, 0x6f, 0xb2, 0xea, 0x82, 0xec, 0xc6, 0x03, 0xfa,
0x84, 0xf3, 0x2a, 0xd4, 0xb1, 0x6d, 0x5b, 0x76, 0xd7, 0xc0, 0x8e, 0xa3, 0xf6, 0x31, 0x4f, 0x80,
0x6b, 0x14, 0xb8, 0xc5, 0x60, 0xf2, 0x8f, 0x79, 0x68, 0xf8, 0x4b, 0xf1, 0xae, 0xf4, 0x74, 0xcd,
0xbb, 0xd2, 0xd3, 0xc9, 0xd1, 0x81, 0xcd, 0x5c, 0xa1, 0x38, 0xdc, 0xd5, 0x5c, 0x4b, 0x52, 0x2a,
0x1c, 0xda, 0xd1, 0x48, 0x5c, 0x25, 0x46, 0x66, 0x5a, 0x1a, 0xf6, 0x0f, 0x17, 0x3c, 0x10, 0x3f,
0xdb, 0x90, 0x8e, 0x14, 0x32, 0xe8, 0x48, 0x31, 0x83, 0x8e, 0x94, 0x12, 0x74, 0x64, 0x11, 0x4a,
0x7b, 0xa3, 0xde, 0x21, 0x76, 0x79, 0xba, 0xca, 0x47, 0x61, 0xdd, 0x29, 0x47, 0x74, 0x47, 0xa8,
0x48, 0x25, 0xa8, 0x22, 0xe7, 0xa0, 0xc2, 0xee, 0x96, 0xba, 0xae, 0x43, 0x3b, 0xe8, 0x79, 0xa5,
0xcc, 0x00, 0xbb, 0x0e, 0xfa, 0xd0, 0xcb, 0xc7, 0xaa, 0x49, 0xc6, 0x4e, 0xbd, 0x4e, 0x44, 0x4b,
0xbc, 0x6c, 0xec, 0x1a, 0x34, 0xe8, 0x4b, 0xf9, 0xe7, 0x23, 0x6c, 0x1f, 0xab, 0x7b, 0x03, 0xdc,
0xaa, 0x51, 0x71, 0xea, 0x04, 0xfa, 0xc4, 0x03, 0x92, 0x0d, 0xa1, 0x68, 0xba, 0xa9, 0xe1, 0x97,
0x58, 0x6b, 0xd5, 0xd9, 0xdd, 0x13, 0x81, 0x75, 0x18, 0x48, 0xfe, 0x02, 0x90, 0x3f, 0xc7, 0x6c,
0x49, 0x59, 0xe4, 0x10, 0x73, 0xd1, 0x43, 0x94, 0xff, 0x56, 0x82, 0x85, 0xe0, 0x64, 0xd3, 0x86,
0xc7, 0x8f, 0xa1, 0xca, 0x6e, 0x1a, 0xba, 0xc4, 0x3c, 0x79, 0xcd, 0x7f, 0x61, 0xec, 0xee, 0x29,
0xe0, 0xbf, 0xd3, 0x26, 0x4a, 0x70, 0x64, 0xd9, 0x87, 0xba, 0xd9, 0xef, 0x12, 0xc9, 0x3c, 0xa3,
0xa8, 0x71, 0xe0, 0x63, 0x02, 0x93, 0xbf, 0x95, 0xe0, 0xe2, 0xd3, 0xa1, 0xa6, 0xba, 0x38, 0x90,
0x27, 0xcc, 0xfa, 0xf4, 0xeb, 0x03, 0xef, 0xed, 0x55, 0x2e, 0x5b, 0xb7, 0x9c, 0x61, 0xcb, 0x5b,
0x70, 0x56, 0xc1, 0x0e, 0x36, 0xb5, 0xd0, 0xc7, 0xa9, 0x2b, 0xfd, 0x21, 0xb4, 0x93, 0xd8, 0xcd,
0x72, 0xf6, 0x2c, 0x61, 0xeb, 0xda, 0x84, 0xad, 0xcb, 0xfd, 0x0f, 0xc9, 0x13, 0xe8, 0x3c, 0xae,
0xfc, 0xdf, 0x12, 0x2c, 0xdc, 0xd7, 0xbc, 0xf9, 0x5e, 0x5b, 0x5e, 0x18, 0xcd, 0x9b, 0xf2, 0xf1,
0xbc, 0xe9, 0x55, 0x39, 0x12, 0xee, 0x52, 0xcd, 0x91, 0xe1, 0x85, 0x0a, 0x9b, 0x5e, 0xed, 0xcb,
0xfb, 0xe2, 0x32, 0x57, 0xc1, 0xfb, 0xd8, 0xc6, 0x66, 0x0f, 0x6f, 0x5a, 0xbd, 0xc3, 0xc0, 0x03,
0x2e, 0x29, 0xf8, 0x80, 0x6b, 0xda, 0x07, 0x61, 0x37, 0xff, 0x42, 0x82, 0x85, 0x58, 0xd7, 0x08,
0x35, 0x00, 0x9e, 0x9a, 0x3d, 0xde, 0x4e, 0x6b, 0x9e, 0x42, 0x35, 0x28, 0x7b, 0xcd, 0xb5, 0xa6,
0x84, 0xaa, 0x30, 0xb7, 0x6b, 0x51, 0xec, 0x66, 0x0e, 0x35, 0xa1, 0xc6, 0x08, 0x47, 0xbd, 0x1e,
0x76, 0x9c, 0x66, 0x5e, 0x40, 0xd6, 0x55, 0x7d, 0x30, 0xb2, 0x71, 0xb3, 0x80, 0xea, 0x50, 0xd9,
0xb5, 0xf8, 0xf3, 0xb7, 0x66, 0x11, 0x21, 0x68, 0x78, 0x6f, 0xe1, 0x38, 0x51, 0x29, 0x00, 0xf3,
0xc8, 0xe6, 0x6e, 0x3e, 0x0b, 0xf6, 0x44, 0x76, 0x8f, 0x87, 0x18, 0x2d, 0xc1, 0xe9, 0xa7, 0xa6,
0x86, 0xf7, 0x75, 0x13, 0x6b, 0xfe, 0xa7, 0xe6, 0x29, 0x74, 0x1a, 0xe6, 0xb7, 0xb0, 0xdd, 0xc7,
0x01, 0x60, 0x0e, 0x2d, 0x40, 0x7d, 0x4b, 0x7f, 0x19, 0x00, 0xe5, 0xe5, 0x42, 0x59, 0x6a, 0x4a,
0x2b, 0xff, 0xb3, 0x04, 0x15, 0x52, 0xcf, 0x3c, 0xb0, 0x2c, 0x5b, 0x43, 0x43, 0x40, 0xf4, 0xb5,
0xa8, 0x31, 0xb4, 0x4c, 0xf1, 0x06, 0x1b, 0xbd, 0x97, 0x92, 0x33, 0xc5, 0x51, 0xb9, 0x1e, 0xb6,
0xaf, 0xa7, 0x50, 0x44, 0xd0, 0xe5, 0x53, 0xc8, 0xa0, 0x33, 0xee, 0xea, 0x06, 0xde, 0xd5, 0x7b,
0x87, 0xde, 0x33, 0x92, 0x31, 0x33, 0x46, 0x50, 0xbd, 0x19, 0x23, 0xd5, 0x33, 0x1f, 0xb0, 0x27,
0xbd, 0x9e, 0x21, 0xca, 0xa7, 0xd0, 0x73, 0x38, 0xb3, 0x81, 0x03, 0x8e, 0xc7, 0x9b, 0x70, 0x25,
0x7d, 0xc2, 0x18, 0xf2, 0x09, 0xa7, 0xdc, 0x84, 0x22, 0x6d, 0xc9, 0xa2, 0x24, 0xdf, 0x14, 0xfc,
0xc9, 0x54, 0xfb, 0x72, 0x3a, 0x82, 0xe0, 0xf6, 0x05, 0xcc, 0x47, 0x7e, 0x68, 0x81, 0x6e, 0x24,
0x90, 0x25, 0xff, 0x64, 0xa6, 0x7d, 0x33, 0x0b, 0xaa, 0x98, 0xab, 0x0f, 0x8d, 0xf0, 0x4b, 0x53,
0x94, 0xd4, 0x14, 0x48, 0x7c, 0x23, 0xdf, 0xbe, 0x91, 0x01, 0x53, 0x4c, 0x64, 0x40, 0x33, 0xfa,
0xf0, 0x1f, 0xdd, 0x1c, 0xcb, 0x20, 0xac, 0x6e, 0xbf, 0xc8, 0x84, 0x2b, 0xa6, 0x3b, 0xa6, 0x4a,
0x10, 0x7b, 0x4b, 0x8e, 0x6e, 0x27, 0xb3, 0x49, 0x7b, 0xe4, 0xde, 0xbe, 0x93, 0x19, 0x5f, 0x4c,
0xfd, 0xbb, 0xec, 0x2a, 0x28, 0xe9, 0x3d, 0x36, 0x7a, 0x3f, 0x99, 0xdd, 0x98, 0x87, 0xe4, 0xed,
0x95, 0x93, 0x90, 0x08, 0x21, 0xbe, 0xa2, 0x77, 0x38, 0x09, 0x2f, 0x9a, 0xa3, 0x76, 0xe7, 0xf1,
0x4b, 0x7f, 0xac, 0xdd, 0x7e, 0xff, 0x04, 0x14, 0x42, 0x00, 0x2b, 0xfa, 0xcb, 0x0a, 0xcf, 0x0c,
0xef, 0x4c, 0xd4, 0x9a, 0xe9, 0x6c, 0xf0, 0x73, 0x98, 0x8f, 0xbc, 0xc6, 0x49, 0xb4, 0x9a, 0xe4,
0x17, 0x3b, 0xed, 0x71, 0xf1, 0x9a, 0x99, 0x64, 0xe4, 0x4a, 0x0c, 0xa5, 0x68, 0x7f, 0xc2, 0xb5,
0x59, 0xfb, 0x66, 0x16, 0x54, 0xb1, 0x10, 0x87, 0xba, 0xcb, 0xc8, 0xb5, 0x12, 0xba, 0x95, 0xcc,
0x23, 0xf9, 0x4a, 0xac, 0xfd, 0x6e, 0x46, 0x6c, 0x31, 0xe9, 0xef, 0x00, 0xda, 0x39, 0x20, 0x35,
0x8c, 0xb9, 0xaf, 0xf7, 0x47, 0xb6, 0xca, 0x9e, 0xdc, 0xa4, 0xf9, 0xe8, 0x38, 0x6a, 0x8a, 0xae,
0x8c, 0xa5, 0x10, 0x93, 0x77, 0x01, 0x36, 0xb0, 0xbb, 0x85, 0x5d, 0x9b, 0x28, 0xe8, 0xf5, 0xc4,
0xf3, 0xf6, 0x11, 0xbc, 0xa9, 0xde, 0x99, 0x88, 0x17, 0x08, 0x09, 0xcd, 0x2d, 0xd5, 0x24, 0xe5,
0xbb, 0xff, 0x52, 0xed, 0x56, 0x22, 0x79, 0x14, 0x2d, 0x65, 0x43, 0x53, 0xb1, 0xc5, 0x94, 0x47,
0x22, 0xcc, 0x06, 0xba, 0xa9, 0x51, 0xf7, 0xe3, 0xcb, 0x9c, 0x7c, 0x21, 0x15, 0x75, 0x3f, 0x63,
0xf0, 0xc5, 0xc4, 0x5f, 0x4b, 0xf4, 0x67, 0x3b, 0x11, 0x84, 0x67, 0xba, 0x7b, 0xb0, 0x3d, 0x50,
0x4d, 0x27, 0x8b, 0x08, 0x14, 0xf1, 0x04, 0x22, 0x70, 0x7c, 0x21, 0x82, 0x06, 0xf5, 0x50, 0x8f,
0x14, 0x25, 0xbd, 0xf9, 0x4a, 0x6a, 0xf8, 0xb6, 0x97, 0x27, 0x23, 0x8a, 0x59, 0x0e, 0xa0, 0xee,
0xa9, 0x34, 0xdb, 0xdc, 0x1b, 0x69, 0x92, 0xfa, 0x38, 0x29, 0x16, 0x99, 0x8c, 0x1a, 0xb4, 0xc8,
0x78, 0x0b, 0x08, 0x65, 0x6b, 0x1d, 0x8e, 0xb3, 0xc8, 0xf4, 0xbe, 0x12, 0x73, 0x39, 0x91, 0x76,
0x6b, 0xb2, 0x3f, 0x4b, 0xec, 0x1e, 0x27, 0xba, 0x9c, 0x94, 0xee, 0xad, 0x7c, 0x0a, 0x3d, 0x83,
0x12, 0xff, 0xbd, 0xee, 0xdb, 0xe3, 0x0b, 0x42, 0xce, 0xfd, 0xda, 0x04, 0x2c, 0xc1, 0xf8, 0x10,
0x96, 0x52, 0xca, 0xc1, 0xc4, 0x50, 0x38, 0xbe, 0x74, 0x9c, 0xe4, 0xa4, 0x55, 0x40, 0xf1, 0x1f,
0xc5, 0x24, 0x1e, 0x53, 0xea, 0x6f, 0x67, 0x32, 0x4c, 0x11, 0xff, 0x5d, 0x4b, 0xe2, 0x14, 0xa9,
0x3f, 0x7f, 0x99, 0x34, 0xc5, 0x13, 0x00, 0xbf, 0xe8, 0x4b, 0x3c, 0x8f, 0x58, 0x4d, 0x38, 0x81,
0xe5, 0xca, 0x7f, 0x94, 0xa1, 0xec, 0xbd, 0xb0, 0x7a, 0x03, 0xf9, 0xff, 0x1b, 0x48, 0xc8, 0x3f,
0x87, 0xf9, 0xc8, 0xaf, 0x39, 0x12, 0x8d, 0x27, 0xf9, 0x17, 0x1f, 0x93, 0x4e, 0xe8, 0x19, 0xff,
0x5f, 0x03, 0x22, 0x36, 0xbf, 0x93, 0x96, 0xd4, 0x47, 0xc3, 0xf2, 0x04, 0xc6, 0xff, 0xbf, 0x83,
0xf0, 0x63, 0x80, 0x40, 0xf8, 0x1d, 0x7f, 0x4f, 0x4e, 0x22, 0xca, 0xa4, 0xdd, 0x32, 0x12, 0x23,
0xec, 0x8d, 0x2c, 0x77, 0x9a, 0xe9, 0x3e, 0x32, 0x3d, 0xae, 0x6e, 0x9d, 0xd0, 0x47, 0x4e, 0x90,
0xde, 0x21, 0x9e, 0x24, 0xda, 0x4e, 0x4a, 0xf1, 0x24, 0x29, 0x4d, 0xac, 0xc4, 0x98, 0x92, 0xde,
0xa3, 0x7a, 0x2d, 0xbe, 0x65, 0xf5, 0xee, 0x6f, 0xbd, 0xdf, 0xd7, 0xdd, 0x83, 0xd1, 0x1e, 0xf9,
0x72, 0x87, 0xa1, 0xbe, 0xab, 0x5b, 0xfc, 0xaf, 0x3b, 0x9e, 0xee, 0xdd, 0xa1, 0xd4, 0x77, 0xc8,
0x1c, 0xc3, 0xbd, 0xbd, 0x12, 0x1d, 0xdd, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xae, 0xee,
0xce, 0x3f, 0xba, 0x44, 0x00, 0x00,
0x39, 0x76, 0x22, 0x37, 0xf8, 0x82, 0xfa, 0xfb, 0xf2, 0xc1, 0xb2, 0x2c, 0x99, 0xad, 0xe4, 0xc8,
0x2b, 0x39, 0x06, 0x9a, 0x02, 0xc4, 0x8a, 0x3b, 0xa2, 0x36, 0xe2, 0xee, 0xd2, 0xbb, 0x4b, 0xcb,
0x4a, 0x1f, 0x12, 0x34, 0x40, 0x81, 0x14, 0x45, 0xd3, 0x0b, 0x8a, 0xb6, 0x0f, 0x05, 0x8a, 0x3e,
0xa5, 0x05, 0x0a, 0x04, 0x08, 0xfa, 0xd0, 0x16, 0x79, 0x0f, 0xda, 0x87, 0xa2, 0x7f, 0xa2, 0xed,
0xaf, 0x28, 0x8a, 0xb9, 0xec, 0xec, 0x9d, 0x5c, 0x91, 0x76, 0x5c, 0x7c, 0x6f, 0x9a, 0xb3, 0xe7,
0x9c, 0x39, 0x33, 0x73, 0xee, 0x33, 0x14, 0x34, 0x35, 0xd5, 0x55, 0xbb, 0x3d, 0xcb, 0xb2, 0xb5,
0xdb, 0x43, 0xdb, 0x72, 0x2d, 0xb4, 0x60, 0xe8, 0x83, 0x17, 0x23, 0x87, 0x8d, 0x6e, 0x93, 0xcf,
0xed, 0x5a, 0xcf, 0x32, 0x0c, 0xcb, 0x64, 0xa0, 0x76, 0x43, 0x37, 0x5d, 0x6c, 0x9b, 0xea, 0x80,
0x8f, 0x6b, 0x41, 0x82, 0x76, 0xcd, 0xe9, 0x1d, 0x60, 0x43, 0x65, 0x23, 0x79, 0x0e, 0x8a, 0x0f,
0x8d, 0xa1, 0x7b, 0x2c, 0xff, 0x95, 0x04, 0xb5, 0xf5, 0xc1, 0xc8, 0x39, 0x50, 0xf0, 0xf3, 0x11,
0x76, 0x5c, 0xf4, 0x1e, 0x14, 0xf6, 0x54, 0x07, 0xb7, 0xa4, 0xcb, 0xd2, 0x72, 0x75, 0xe5, 0xfc,
0xed, 0xd0, 0xac, 0x7c, 0xbe, 0x2d, 0xa7, 0xbf, 0xaa, 0x3a, 0x58, 0xa1, 0x98, 0x08, 0x41, 0x41,
0xdb, 0xeb, 0xac, 0xb5, 0x72, 0x97, 0xa5, 0xe5, 0xbc, 0x42, 0xff, 0x46, 0x17, 0x01, 0x1c, 0xdc,
0x37, 0xb0, 0xe9, 0x76, 0xd6, 0x9c, 0x56, 0xfe, 0x72, 0x7e, 0x39, 0xaf, 0x04, 0x20, 0x48, 0x86,
0x5a, 0xcf, 0x1a, 0x0c, 0x70, 0xcf, 0xd5, 0x2d, 0xb3, 0xb3, 0xd6, 0x2a, 0x50, 0xda, 0x10, 0x4c,
0xfe, 0x2f, 0x09, 0xea, 0x5c, 0x34, 0x67, 0x68, 0x99, 0x0e, 0x46, 0x77, 0xa1, 0xe4, 0xb8, 0xaa,
0x3b, 0x72, 0xb8, 0x74, 0xe7, 0x12, 0xa5, 0xdb, 0xa1, 0x28, 0x0a, 0x47, 0x4d, 0x14, 0x2f, 0x3a,
0x7d, 0x3e, 0x3e, 0x7d, 0x64, 0x09, 0x85, 0xd8, 0x12, 0x96, 0x61, 0x7e, 0x9f, 0x48, 0xb7, 0xe3,
0x23, 0x15, 0x29, 0x52, 0x14, 0x4c, 0x38, 0xb9, 0xba, 0x81, 0x3f, 0xde, 0xdf, 0xc1, 0xea, 0xa0,
0x55, 0xa2, 0x73, 0x05, 0x20, 0xf2, 0x7f, 0x4a, 0xd0, 0x14, 0xe8, 0xde, 0x39, 0x9c, 0x81, 0x62,
0xcf, 0x1a, 0x99, 0x2e, 0x5d, 0x6a, 0x5d, 0x61, 0x03, 0x74, 0x05, 0x6a, 0xbd, 0x03, 0xd5, 0x34,
0xf1, 0xa0, 0x6b, 0xaa, 0x06, 0xa6, 0x8b, 0xaa, 0x28, 0x55, 0x0e, 0x7b, 0xac, 0x1a, 0x38, 0xd3,
0xda, 0x2e, 0x43, 0x75, 0xa8, 0xda, 0xae, 0x1e, 0xda, 0xfd, 0x20, 0x08, 0xb5, 0xa1, 0xac, 0x3b,
0x1d, 0x63, 0x68, 0xd9, 0x6e, 0xab, 0x78, 0x59, 0x5a, 0x2e, 0x2b, 0x62, 0x4c, 0x66, 0xd0, 0xe9,
0x5f, 0xbb, 0xaa, 0x73, 0xd8, 0x59, 0xe3, 0x2b, 0x0a, 0xc1, 0xe4, 0xbf, 0x95, 0x60, 0xf1, 0xbe,
0xe3, 0xe8, 0x7d, 0x33, 0xb6, 0xb2, 0x45, 0x28, 0x99, 0x96, 0x86, 0x3b, 0x6b, 0x74, 0x69, 0x79,
0x85, 0x8f, 0xd0, 0x39, 0xa8, 0x0c, 0x31, 0xb6, 0xbb, 0xb6, 0x35, 0xf0, 0x16, 0x56, 0x26, 0x00,
0xc5, 0x1a, 0x60, 0xf4, 0x04, 0x16, 0x9c, 0x08, 0x23, 0xa6, 0x57, 0xd5, 0x95, 0xab, 0xb7, 0x63,
0x96, 0x71, 0x3b, 0x3a, 0xa9, 0x12, 0xa7, 0x96, 0xbf, 0xcc, 0xc1, 0x69, 0x81, 0xc7, 0x64, 0x25,
0x7f, 0x93, 0x9d, 0x77, 0x70, 0x5f, 0x88, 0xc7, 0x06, 0x59, 0x76, 0x5e, 0x1c, 0x59, 0x3e, 0x78,
0x64, 0x19, 0x54, 0x3d, 0x7a, 0x1e, 0xc5, 0xf8, 0x79, 0x5c, 0x82, 0x2a, 0x7e, 0x39, 0xd4, 0x6d,
0xdc, 0x25, 0x8a, 0x43, 0xb7, 0xbc, 0xa0, 0x00, 0x03, 0xed, 0xea, 0x46, 0xd0, 0x36, 0xe6, 0x32,
0xdb, 0x86, 0xfc, 0x77, 0x12, 0x2c, 0xc5, 0x4e, 0x89, 0x1b, 0x9b, 0x02, 0x4d, 0xba, 0x72, 0x7f,
0x67, 0x88, 0xd9, 0x91, 0x0d, 0xbf, 0x3e, 0x6e, 0xc3, 0x7d, 0x74, 0x25, 0x46, 0x1f, 0x10, 0x32,
0x97, 0x5d, 0xc8, 0x43, 0x58, 0xda, 0xc0, 0x2e, 0x9f, 0x80, 0x7c, 0xc3, 0xce, 0xf4, 0xce, 0x2a,
0x6c, 0xd5, 0xb9, 0xa8, 0x55, 0xcb, 0xdf, 0xe5, 0x84, 0x2d, 0xd2, 0xa9, 0x3a, 0xe6, 0xbe, 0x85,
0xce, 0x43, 0x45, 0xa0, 0x70, 0xad, 0xf0, 0x01, 0xe8, 0xe7, 0x50, 0x24, 0x92, 0x32, 0x95, 0x68,
0xac, 0x5c, 0x49, 0x5e, 0x53, 0x80, 0xa7, 0xc2, 0xf0, 0x51, 0x07, 0x1a, 0x8e, 0xab, 0xda, 0x6e,
0x77, 0x68, 0x39, 0xf4, 0x9c, 0xa9, 0xe2, 0x54, 0x57, 0xe4, 0x30, 0x07, 0xe1, 0xd6, 0xb7, 0x9c,
0xfe, 0x36, 0xc7, 0x54, 0xea, 0x94, 0xd2, 0x1b, 0xa2, 0x87, 0x50, 0xc3, 0xa6, 0xe6, 0x33, 0x2a,
0x64, 0x66, 0x54, 0xc5, 0xa6, 0x26, 0xd8, 0xf8, 0xe7, 0x53, 0xcc, 0x7e, 0x3e, 0x7f, 0x24, 0x41,
0x2b, 0x7e, 0x40, 0xb3, 0xb8, 0xec, 0x7b, 0x8c, 0x08, 0xb3, 0x03, 0x1a, 0x6b, 0xe1, 0xe2, 0x90,
0x14, 0x4e, 0x22, 0xff, 0x85, 0x04, 0x6f, 0xf9, 0xe2, 0xd0, 0x4f, 0xaf, 0x4b, 0x5b, 0xd0, 0x4d,
0x68, 0xea, 0x66, 0x6f, 0x30, 0xd2, 0xf0, 0x53, 0xf3, 0x11, 0x56, 0x07, 0xee, 0xc1, 0x31, 0x3d,
0xc3, 0xb2, 0x12, 0x83, 0xcb, 0x5f, 0x49, 0xb0, 0x18, 0x95, 0x6b, 0x96, 0x4d, 0xfa, 0x0d, 0x28,
0xea, 0xe6, 0xbe, 0xe5, 0xed, 0xd1, 0xc5, 0x31, 0x46, 0x49, 0xe6, 0x62, 0xc8, 0xb2, 0x01, 0xe7,
0x36, 0xb0, 0xdb, 0x31, 0x1d, 0x6c, 0xbb, 0xab, 0xba, 0x39, 0xb0, 0xfa, 0xdb, 0xaa, 0x7b, 0x30,
0x83, 0x41, 0x85, 0x6c, 0x23, 0x17, 0xb1, 0x0d, 0xf9, 0x5b, 0x09, 0xce, 0x27, 0xcf, 0xc7, 0x97,
0xde, 0x86, 0xf2, 0xbe, 0x8e, 0x07, 0x1a, 0xd9, 0x5f, 0x89, 0xee, 0xaf, 0x18, 0x13, 0xc3, 0x1a,
0x12, 0x64, 0xbe, 0xc2, 0x2b, 0x29, 0xda, 0xbc, 0xe3, 0xda, 0xba, 0xd9, 0xdf, 0xd4, 0x1d, 0x57,
0x61, 0xf8, 0x81, 0xfd, 0xcc, 0x67, 0x57, 0xe3, 0x3f, 0x94, 0xe0, 0xe2, 0x06, 0x76, 0x1f, 0x08,
0xbf, 0x4c, 0xbe, 0xeb, 0x8e, 0xab, 0xf7, 0x9c, 0x57, 0x9b, 0x1b, 0x65, 0x08, 0xd0, 0xf2, 0x37,
0x12, 0x5c, 0x4a, 0x15, 0x86, 0x6f, 0x1d, 0xf7, 0x3b, 0x9e, 0x57, 0x4e, 0xf6, 0x3b, 0xbf, 0x8d,
0x8f, 0x3f, 0x51, 0x07, 0x23, 0xbc, 0xad, 0xea, 0x36, 0xf3, 0x3b, 0x53, 0x7a, 0xe1, 0x7f, 0x94,
0xe0, 0xc2, 0x06, 0x76, 0xb7, 0xbd, 0x98, 0xf4, 0x06, 0x77, 0x87, 0xe0, 0x04, 0x62, 0xa3, 0x97,
0x9c, 0x85, 0x60, 0xf2, 0x1f, 0xb3, 0xe3, 0x4c, 0x94, 0xf7, 0x8d, 0x6c, 0xe0, 0x45, 0x6a, 0x09,
0x01, 0x93, 0x7c, 0xc0, 0x52, 0x07, 0xbe, 0x7d, 0xf2, 0xdf, 0x48, 0x70, 0xf6, 0x7e, 0xef, 0xf9,
0x48, 0xb7, 0x31, 0x47, 0xda, 0xb4, 0x7a, 0x87, 0xd3, 0x6f, 0xae, 0x9f, 0x66, 0xe5, 0x42, 0x69,
0xd6, 0xa4, 0xd4, 0x7c, 0x11, 0x4a, 0x2e, 0xcb, 0xeb, 0x58, 0xa6, 0xc2, 0x47, 0x54, 0x3e, 0x05,
0x0f, 0xb0, 0xea, 0xfc, 0xff, 0x94, 0xef, 0x9b, 0x02, 0xd4, 0x3e, 0xe1, 0xe9, 0x18, 0x8d, 0xda,
0x51, 0x4d, 0x92, 0x92, 0x13, 0xaf, 0x40, 0x06, 0x97, 0x94, 0xd4, 0x6d, 0x40, 0xdd, 0xc1, 0xf8,
0x70, 0x9a, 0x18, 0x5d, 0x23, 0x84, 0x22, 0xb6, 0x6e, 0xc2, 0xc2, 0xc8, 0xa4, 0xa5, 0x01, 0xd6,
0xf8, 0x06, 0x32, 0xcd, 0x9d, 0xec, 0xbb, 0xe3, 0x84, 0xe8, 0x11, 0xaf, 0x3e, 0x02, 0xbc, 0x8a,
0x99, 0x78, 0x45, 0xc9, 0x50, 0x07, 0x9a, 0x9a, 0x6d, 0x0d, 0x87, 0x58, 0xeb, 0x3a, 0x1e, 0xab,
0x52, 0x36, 0x56, 0x9c, 0x4e, 0xb0, 0x7a, 0x0f, 0x4e, 0x47, 0x25, 0xed, 0x68, 0x24, 0x21, 0x25,
0x67, 0x98, 0xf4, 0x09, 0xdd, 0x82, 0x85, 0x38, 0x7e, 0x99, 0xe2, 0xc7, 0x3f, 0xa0, 0x77, 0x01,
0x45, 0x44, 0x25, 0xe8, 0x15, 0x86, 0x1e, 0x16, 0xa6, 0xa3, 0x39, 0xf2, 0xd7, 0x12, 0x2c, 0x3e,
0x53, 0xdd, 0xde, 0xc1, 0x9a, 0xc1, 0x6d, 0x6d, 0x06, 0x5f, 0xf5, 0x4b, 0xa8, 0xbc, 0xe0, 0x7a,
0xe1, 0x05, 0xa4, 0x4b, 0x09, 0xfb, 0x13, 0xd4, 0x40, 0xc5, 0xa7, 0x90, 0x7f, 0x94, 0xe0, 0xcc,
0x7a, 0xa0, 0x2e, 0x7c, 0x03, 0x5e, 0x73, 0x52, 0x41, 0x7b, 0x1d, 0x1a, 0x86, 0x6a, 0x1f, 0xc6,
0xea, 0xd9, 0x08, 0x54, 0x7e, 0x09, 0xc0, 0x47, 0x5b, 0x4e, 0x7f, 0x0a, 0xf9, 0x3f, 0x84, 0x39,
0x3e, 0x2b, 0x77, 0x9f, 0x93, 0xf4, 0xcc, 0x43, 0x97, 0xff, 0x4d, 0x82, 0x86, 0x1f, 0x12, 0xa9,
0x91, 0x37, 0x20, 0x27, 0x4c, 0x3b, 0xd7, 0x59, 0x43, 0xbf, 0x84, 0x12, 0x6b, 0x74, 0x70, 0xde,
0xd7, 0xc2, 0xbc, 0x79, 0x13, 0x24, 0x10, 0x57, 0x29, 0x40, 0xe1, 0x44, 0x64, 0x8f, 0x44, 0x14,
0x11, 0xce, 0xc7, 0x87, 0xa0, 0x0e, 0xcc, 0x87, 0x53, 0x76, 0xcf, 0x84, 0x2f, 0xa7, 0x05, 0x8f,
0x35, 0xd5, 0x55, 0x69, 0xec, 0x68, 0x84, 0x32, 0x76, 0x47, 0xfe, 0xd3, 0x12, 0x54, 0x03, 0xab,
0x8c, 0xad, 0x24, 0x7a, 0xa4, 0xb9, 0xc9, 0x75, 0x63, 0x3e, 0x5e, 0x37, 0x5e, 0x83, 0x86, 0x4e,
0x93, 0xaf, 0x2e, 0x57, 0x45, 0xea, 0x35, 0x2b, 0x4a, 0x9d, 0x41, 0xb9, 0x5d, 0xa0, 0x8b, 0x50,
0x35, 0x47, 0x46, 0xd7, 0xda, 0xef, 0xda, 0xd6, 0x91, 0xc3, 0x0b, 0xd0, 0x8a, 0x39, 0x32, 0x3e,
0xde, 0x57, 0xac, 0x23, 0xc7, 0xaf, 0x71, 0x4a, 0x27, 0xac, 0x71, 0x2e, 0x42, 0xd5, 0x50, 0x5f,
0x12, 0xae, 0x5d, 0x73, 0x64, 0xd0, 0xda, 0x34, 0xaf, 0x54, 0x0c, 0xf5, 0xa5, 0x62, 0x1d, 0x3d,
0x1e, 0x19, 0x68, 0x19, 0x9a, 0x03, 0xd5, 0x71, 0xbb, 0xc1, 0xe2, 0xb6, 0x4c, 0x8b, 0xdb, 0x06,
0x81, 0x3f, 0xf4, 0x0b, 0xdc, 0x78, 0xb5, 0x54, 0x99, 0xa1, 0x5a, 0xd2, 0x8c, 0x81, 0xcf, 0x08,
0xb2, 0x57, 0x4b, 0x9a, 0x31, 0x10, 0x6c, 0x3e, 0x84, 0xb9, 0x3d, 0x9a, 0xd2, 0x3a, 0xad, 0x6a,
0xaa, 0xc3, 0x5c, 0x27, 0xd9, 0x2c, 0xcb, 0x7c, 0x15, 0x0f, 0x1d, 0xfd, 0x02, 0x2a, 0x34, 0x93,
0xa0, 0xb4, 0xb5, 0x4c, 0xb4, 0x3e, 0x01, 0xa1, 0xd6, 0xf0, 0xc0, 0x55, 0x29, 0x75, 0x3d, 0x1b,
0xb5, 0x20, 0x20, 0x4e, 0xba, 0x67, 0x63, 0xd5, 0xc5, 0xda, 0xea, 0xf1, 0x03, 0xcb, 0x18, 0xaa,
0x54, 0x99, 0x5a, 0x0d, 0x5a, 0xb6, 0x24, 0x7d, 0x22, 0x8e, 0xa1, 0x27, 0x46, 0xeb, 0xb6, 0x65,
0xb4, 0xe6, 0x99, 0x63, 0x08, 0x43, 0xd1, 0x05, 0x00, 0xcf, 0x3d, 0xab, 0x6e, 0xab, 0x49, 0x4f,
0xb1, 0xc2, 0x21, 0xf7, 0x69, 0xef, 0x4a, 0x77, 0xba, 0xac, 0x4b, 0xa4, 0x9b, 0xfd, 0xd6, 0x02,
0x9d, 0xb1, 0xea, 0xb5, 0x95, 0x74, 0xb3, 0x2f, 0x7f, 0x01, 0x67, 0x7c, 0x25, 0x0a, 0x1c, 0x58,
0xfc, 0xec, 0xa5, 0x69, 0xcf, 0x7e, 0x7c, 0xbd, 0xf2, 0x1f, 0x05, 0x58, 0xdc, 0x51, 0x5f, 0xe0,
0xd7, 0x5f, 0x1a, 0x65, 0x72, 0xd9, 0x9b, 0xb0, 0x40, 0xab, 0xa1, 0x95, 0x80, 0x3c, 0x63, 0x72,
0x86, 0xe0, 0x89, 0xc7, 0x09, 0xd1, 0xaf, 0x48, 0xb2, 0x83, 0x7b, 0x87, 0xdb, 0x96, 0xee, 0xe7,
0x0b, 0x17, 0x12, 0xf8, 0x3c, 0x10, 0x58, 0x4a, 0x90, 0x02, 0x6d, 0xc7, 0xbd, 0x1f, 0xcb, 0x14,
0xde, 0x19, 0x5b, 0xa0, 0xfb, 0xbb, 0x1f, 0x75, 0x82, 0xa8, 0x05, 0x73, 0x3c, 0xcc, 0x53, 0xd7,
0x50, 0x56, 0xbc, 0x21, 0xda, 0x86, 0xd3, 0x6c, 0x05, 0x3b, 0x5c, 0xef, 0xd9, 0xe2, 0xcb, 0x99,
0x16, 0x9f, 0x44, 0x1a, 0x36, 0x9b, 0xca, 0x49, 0xcd, 0xa6, 0x05, 0x73, 0x5c, 0x95, 0xa9, 0xbb,
0x28, 0x2b, 0xde, 0x90, 0x1c, 0xb3, 0xaf, 0xd4, 0x55, 0xfa, 0xcd, 0x07, 0x90, 0xb2, 0x12, 0xfc,
0xfd, 0x9c, 0xd0, 0x4a, 0xfa, 0x08, 0xca, 0x42, 0xc3, 0x73, 0x99, 0x35, 0x5c, 0xd0, 0x44, 0xdd,
0x78, 0x3e, 0xe2, 0xc6, 0xe5, 0x7f, 0x97, 0xa0, 0xb6, 0x46, 0x96, 0xb4, 0x69, 0xf5, 0x69, 0xd0,
0xb9, 0x06, 0x0d, 0x1b, 0xf7, 0x2c, 0x5b, 0xeb, 0x62, 0xd3, 0xb5, 0x75, 0xcc, 0x3a, 0x10, 0x05,
0xa5, 0xce, 0xa0, 0x0f, 0x19, 0x90, 0xa0, 0x11, 0xcf, 0xec, 0xb8, 0xaa, 0x31, 0xec, 0xee, 0x13,
0x0f, 0x90, 0x63, 0x68, 0x02, 0x4a, 0x1d, 0xc0, 0x15, 0xa8, 0xf9, 0x68, 0xae, 0x45, 0xe7, 0x2f,
0x28, 0x55, 0x01, 0xdb, 0xb5, 0xd0, 0xdb, 0xd0, 0xa0, 0x7b, 0xda, 0x1d, 0x58, 0xfd, 0x2e, 0xa9,
0xd6, 0x79, 0x3c, 0xaa, 0x69, 0x5c, 0x2c, 0x72, 0x56, 0x61, 0x2c, 0x47, 0xff, 0x1c, 0xf3, 0x88,
0x24, 0xb0, 0x76, 0xf4, 0xcf, 0x31, 0x49, 0x07, 0xea, 0x24, 0xbc, 0x3e, 0xb6, 0x34, 0xbc, 0x3b,
0x65, 0x32, 0x92, 0xa1, 0xad, 0x7b, 0x1e, 0x2a, 0x62, 0x05, 0x7c, 0x49, 0x3e, 0x00, 0xad, 0x43,
0xc3, 0x4b, 0x9b, 0xbb, 0xac, 0x9a, 0x2c, 0xa4, 0x26, 0x87, 0x81, 0x00, 0xe9, 0x28, 0x75, 0x8f,
0x8c, 0x0e, 0xe5, 0x75, 0xa8, 0x05, 0x3f, 0x93, 0x59, 0x77, 0xa2, 0x8a, 0x22, 0x00, 0x44, 0x1b,
0x1f, 0x8f, 0x0c, 0x72, 0xa6, 0xdc, 0xb1, 0x78, 0x43, 0xf9, 0x2b, 0x09, 0xea, 0x3c, 0xaa, 0xef,
0x88, 0x0b, 0x10, 0xba, 0x34, 0x89, 0x2e, 0x8d, 0xfe, 0x8d, 0x7e, 0x33, 0xdc, 0xb3, 0x7c, 0x3b,
0xd1, 0x09, 0x50, 0x26, 0x34, 0x81, 0x0e, 0x85, 0xf4, 0x2c, 0xfd, 0x8b, 0x2f, 0x89, 0xa2, 0xf1,
0xa3, 0xa1, 0x8a, 0xd6, 0x82, 0x39, 0x55, 0xd3, 0x6c, 0xec, 0x38, 0x5c, 0x0e, 0x6f, 0x48, 0xbe,
0xbc, 0xc0, 0xb6, 0xe3, 0xa9, 0x7c, 0x5e, 0xf1, 0x86, 0xe8, 0x17, 0x50, 0x16, 0x19, 0x77, 0x3e,
0x29, 0xcb, 0x0a, 0xca, 0xc9, 0xab, 0x6d, 0x41, 0x21, 0xff, 0x53, 0x0e, 0x1a, 0x7c, 0xc3, 0x56,
0x79, 0xd8, 0x1d, 0x6f, 0x7c, 0xab, 0x50, 0xdb, 0xf7, 0x6d, 0x7f, 0x5c, 0x5f, 0x2d, 0xe8, 0x22,
0x42, 0x34, 0x93, 0x0c, 0x30, 0x1c, 0xf8, 0x0b, 0x33, 0x05, 0xfe, 0xe2, 0x49, 0x3d, 0x58, 0x3c,
0x15, 0x2c, 0x25, 0xa4, 0x82, 0xf2, 0xef, 0x42, 0x35, 0xc0, 0x80, 0x7a, 0x68, 0xd6, 0x90, 0xe3,
0x3b, 0xe6, 0x0d, 0xd1, 0x5d, 0x3f, 0xfd, 0x61, 0x5b, 0x75, 0x36, 0x41, 0x96, 0x48, 0xe6, 0x23,
0xff, 0xbd, 0x04, 0x25, 0xce, 0xf9, 0x12, 0x54, 0xb9, 0xd3, 0xa1, 0xa9, 0x21, 0xe3, 0x0e, 0x1c,
0x44, 0x72, 0xc3, 0x57, 0xe7, 0x75, 0xce, 0x42, 0x39, 0xe2, 0x6f, 0xe6, 0x78, 0x58, 0xf0, 0x3e,
0x05, 0x9c, 0x0c, 0xf9, 0x44, 0xfd, 0xcb, 0x8f, 0x12, 0xbd, 0x79, 0x50, 0x70, 0xcf, 0x7a, 0x81,
0xed, 0xe3, 0xd9, 0x5b, 0xb6, 0xf7, 0x02, 0x0a, 0x9d, 0xb1, 0x84, 0x14, 0x04, 0xe8, 0x9e, 0xbf,
0xdd, 0xf9, 0xa4, 0x7e, 0x55, 0xd0, 0xc3, 0x70, 0x75, 0xf4, 0xb7, 0xfd, 0x4f, 0x58, 0xf3, 0x39,
0xbc, 0x94, 0x69, 0xf3, 0x9a, 0x57, 0x52, 0x99, 0xc8, 0x7f, 0x2e, 0xc1, 0xd9, 0x0d, 0xec, 0xae,
0x87, 0xdb, 0x11, 0x6f, 0x5a, 0x2a, 0x03, 0xda, 0x49, 0x42, 0xcd, 0x72, 0xea, 0x6d, 0x28, 0x8b,
0xc6, 0x0a, 0xbb, 0x42, 0x10, 0x63, 0xf9, 0x0f, 0x24, 0x68, 0xf1, 0x59, 0xe8, 0x9c, 0x24, 0xeb,
0x1e, 0x60, 0x17, 0x6b, 0x3f, 0x75, 0x69, 0xfd, 0x83, 0x04, 0xcd, 0xa0, 0xc7, 0xa7, 0x4e, 0xfb,
0x03, 0x28, 0xd2, 0x0e, 0x06, 0x97, 0x60, 0xa2, 0xb2, 0x32, 0x6c, 0xe2, 0x32, 0x68, 0x9a, 0xb7,
0x2b, 0x82, 0x13, 0x1f, 0xfa, 0x61, 0x27, 0x7f, 0xf2, 0xb0, 0xc3, 0xc3, 0xb0, 0x35, 0x22, 0x7c,
0x59, 0xeb, 0xcf, 0x07, 0xc8, 0xbf, 0x05, 0x8b, 0x7e, 0xc5, 0xc2, 0xe8, 0xa6, 0xd5, 0x24, 0xf9,
0x2f, 0x25, 0x38, 0xbd, 0x73, 0x6c, 0xf6, 0xa2, 0x3a, 0xb9, 0x08, 0xa5, 0xe1, 0x40, 0xf5, 0x5b,
0x89, 0x7c, 0x44, 0x73, 0x08, 0x36, 0x37, 0xd6, 0x88, 0x03, 0x62, 0x8b, 0xae, 0x0a, 0xd8, 0xae,
0x35, 0x31, 0x2e, 0x5c, 0x13, 0x25, 0x16, 0xd6, 0x98, 0xab, 0x63, 0xfd, 0x99, 0xba, 0x80, 0x12,
0x57, 0x27, 0x7f, 0x9f, 0x83, 0x56, 0x60, 0x99, 0x3f, 0x75, 0x74, 0x4b, 0xc9, 0xc9, 0xf3, 0xaf,
0x28, 0x27, 0x2f, 0xcc, 0x1e, 0xd1, 0x8a, 0x49, 0x11, 0xed, 0x5f, 0x73, 0xd0, 0xf0, 0x77, 0x6d,
0x7b, 0xa0, 0x9a, 0xa9, 0x47, 0xb9, 0x23, 0xb2, 0xb9, 0xf0, 0x3e, 0xfd, 0x2c, 0x49, 0x53, 0x53,
0x0e, 0x42, 0x89, 0xb0, 0x20, 0x75, 0x31, 0x2b, 0x9b, 0x68, 0x77, 0x83, 0x67, 0x90, 0xcc, 0x24,
0x74, 0x03, 0xa3, 0x5b, 0x80, 0xb8, 0x1e, 0x77, 0x75, 0xb3, 0xeb, 0xe0, 0x9e, 0x65, 0x6a, 0x4c,
0xc3, 0x8b, 0x4a, 0x93, 0x7f, 0xe9, 0x98, 0x3b, 0x0c, 0x8e, 0x3e, 0x80, 0x82, 0x7b, 0x3c, 0x64,
0xb1, 0xaa, 0x91, 0x18, 0x03, 0x7c, 0xb9, 0x76, 0x8f, 0x87, 0x58, 0xa1, 0xe8, 0xde, 0x1b, 0x14,
0xd7, 0x56, 0x5f, 0xf0, 0xc0, 0x5f, 0x50, 0x02, 0x10, 0x62, 0xb3, 0xde, 0x1e, 0xce, 0xb1, 0x00,
0xc9, 0x87, 0xf2, 0x3f, 0xe7, 0xa0, 0xe9, 0xb3, 0x54, 0xb0, 0x33, 0x1a, 0xa4, 0x9b, 0xc2, 0xf8,
0x92, 0x77, 0x92, 0x15, 0xfc, 0x0a, 0xaa, 0xfc, 0x3c, 0x4f, 0xa0, 0x0f, 0xc0, 0x48, 0x36, 0xc7,
0x28, 0x68, 0xf1, 0x15, 0x29, 0x68, 0xe9, 0x84, 0x0a, 0x2a, 0x7f, 0x2b, 0xc1, 0x5b, 0x31, 0xb7,
0x34, 0x76, 0x03, 0xc7, 0x27, 0xe6, 0xdc, 0x5d, 0x45, 0x59, 0x72, 0x0f, 0x79, 0x0f, 0x4a, 0x36,
0xe5, 0xce, 0xef, 0x28, 0xae, 0x8e, 0x55, 0x0e, 0x26, 0x88, 0xc2, 0x49, 0xe4, 0x3f, 0x93, 0x60,
0x29, 0x2e, 0xea, 0x0c, 0x61, 0x6f, 0x15, 0xe6, 0x18, 0x6b, 0xcf, 0x86, 0x96, 0xc7, 0xdb, 0x90,
0xbf, 0x39, 0x8a, 0x47, 0x28, 0xef, 0xc0, 0xa2, 0x17, 0x1d, 0xfd, 0x0d, 0xde, 0xc2, 0xae, 0x3a,
0x26, 0x2d, 0xbd, 0x04, 0x55, 0x96, 0xf5, 0xb0, 0x74, 0x8f, 0x15, 0x74, 0xb0, 0x27, 0xfa, 0x20,
0x24, 0x05, 0x3d, 0x43, 0xc3, 0x4b, 0xf4, 0x52, 0x20, 0xcb, 0x85, 0x91, 0x2c, 0xea, 0x45, 0x52,
0x1b, 0xb2, 0xa5, 0x55, 0x94, 0x10, 0x2c, 0xa9, 0x49, 0x9c, 0x9f, 0xb2, 0x49, 0xbc, 0x09, 0x6f,
0x45, 0x44, 0x9d, 0xe1, 0x48, 0xc8, 0xca, 0x17, 0x77, 0xc2, 0x2f, 0x35, 0xa6, 0xcf, 0xb7, 0x2e,
0x88, 0xeb, 0x84, 0xae, 0xae, 0x45, 0x6d, 0x5d, 0x43, 0x1f, 0x41, 0xc5, 0xc4, 0x47, 0xdd, 0x60,
0xb8, 0xcf, 0xd0, 0x35, 0x2e, 0x9b, 0xf8, 0x88, 0xfe, 0x25, 0x3f, 0x86, 0xa5, 0x98, 0xa8, 0xb3,
0xac, 0xfd, 0x5f, 0x24, 0x38, 0xbb, 0x66, 0x5b, 0xc3, 0x4f, 0x74, 0xdb, 0x1d, 0xa9, 0x83, 0xf0,
0xe5, 0xeb, 0xeb, 0x69, 0x1c, 0x3c, 0x0a, 0x24, 0x7e, 0x4c, 0x01, 0x6e, 0x25, 0x98, 0x40, 0x5c,
0x28, 0xbe, 0xe8, 0x40, 0x9a, 0xf8, 0xdf, 0xf9, 0x24, 0xe1, 0x39, 0xde, 0x84, 0xc0, 0x9f, 0x25,
0x2f, 0x4e, 0xec, 0x33, 0xe6, 0xa7, 0xed, 0x33, 0xa6, 0x78, 0xe1, 0xc2, 0x2b, 0xf2, 0xc2, 0x27,
0x2e, 0x7c, 0x1f, 0x41, 0xb8, 0x07, 0x4c, 0xc3, 0xdf, 0x54, 0xcd, 0xe3, 0x55, 0x00, 0xbf, 0x1f,
0xca, 0x1f, 0xda, 0x65, 0x61, 0x13, 0xa0, 0x22, 0xa7, 0x25, 0x22, 0x1e, 0xbd, 0xea, 0x08, 0x75,
0xe8, 0x9e, 0x40, 0x3b, 0x49, 0x4b, 0x67, 0xd1, 0xfc, 0xef, 0x73, 0x00, 0x1d, 0xf1, 0x36, 0x73,
0x3a, 0x67, 0x7e, 0x15, 0xea, 0xbe, 0xc2, 0xf8, 0xf6, 0x1e, 0xd4, 0x22, 0x8d, 0x98, 0x84, 0x28,
0xa5, 0x08, 0x4e, 0xac, 0xbc, 0xd2, 0x28, 0x9f, 0x80, 0xd5, 0x30, 0xa5, 0x88, 0xfa, 0xcf, 0x73,
0x50, 0xb1, 0xad, 0xa3, 0x2e, 0x31, 0x33, 0xcd, 0x7b, 0x7c, 0x6a, 0x5b, 0x47, 0xc4, 0xf8, 0x34,
0xb4, 0x04, 0x73, 0xae, 0xea, 0x1c, 0x12, 0xfe, 0xa5, 0xc0, 0xfd, 0xbf, 0x86, 0xce, 0x40, 0x71,
0x5f, 0x1f, 0x60, 0x76, 0xdd, 0x5c, 0x51, 0xd8, 0x00, 0xfd, 0xdc, 0x7b, 0x25, 0x55, 0xce, 0xfc,
0xc6, 0x83, 0x3d, 0x94, 0xfa, 0x51, 0x82, 0x79, 0x7f, 0xd7, 0xa8, 0x03, 0x22, 0x3e, 0x8d, 0xfa,
0xb3, 0x07, 0x96, 0xc6, 0x5c, 0x45, 0x23, 0xc5, 0xa5, 0x33, 0x42, 0xe6, 0xb5, 0x7c, 0x92, 0x71,
0x95, 0x20, 0x59, 0x17, 0x59, 0xb4, 0xae, 0x79, 0xd7, 0x8e, 0x25, 0xdb, 0x3a, 0xea, 0x68, 0x62,
0x37, 0xd8, 0xcb, 0x52, 0x56, 0xf7, 0x90, 0xdd, 0x78, 0x40, 0x1f, 0x97, 0x5e, 0x85, 0x3a, 0xb6,
0x6d, 0xcb, 0xee, 0x1a, 0xd8, 0x71, 0xd4, 0x3e, 0xe6, 0x09, 0x70, 0x8d, 0x02, 0xb7, 0x18, 0x4c,
0xfe, 0x21, 0x0f, 0x0d, 0x7f, 0x29, 0xde, 0x65, 0xa3, 0xae, 0x79, 0x97, 0x8d, 0x3a, 0x39, 0x3a,
0xb0, 0x99, 0x2b, 0x14, 0x87, 0xbb, 0x9a, 0x6b, 0x49, 0x4a, 0x85, 0x43, 0x3b, 0x1a, 0x89, 0xab,
0xc4, 0xc8, 0x4c, 0x4b, 0xc3, 0xfe, 0xe1, 0x82, 0x07, 0xe2, 0x67, 0x1b, 0xd2, 0x91, 0x42, 0x06,
0x1d, 0x29, 0x66, 0xd0, 0x91, 0x52, 0x82, 0x8e, 0x2c, 0x42, 0x69, 0x6f, 0xd4, 0x3b, 0xc4, 0x2e,
0x4f, 0x57, 0xf9, 0x28, 0xac, 0x3b, 0xe5, 0x88, 0xee, 0x08, 0x15, 0xa9, 0x04, 0x55, 0xe4, 0x1c,
0x54, 0xd8, 0xad, 0x57, 0xd7, 0x75, 0x68, 0x6f, 0x3f, 0xaf, 0x94, 0x19, 0x60, 0xd7, 0x41, 0x1f,
0x7a, 0xf9, 0x58, 0x35, 0xc9, 0xd8, 0xa9, 0xd7, 0x89, 0x68, 0x89, 0x97, 0x8d, 0x5d, 0x83, 0x06,
0x7d, 0xc3, 0xff, 0x7c, 0x84, 0xed, 0x63, 0x75, 0x6f, 0x80, 0x5b, 0x35, 0x2a, 0x4e, 0x9d, 0x40,
0x9f, 0x78, 0x40, 0xb2, 0x21, 0x14, 0x4d, 0x37, 0x35, 0xfc, 0x12, 0x6b, 0xad, 0x3a, 0xbb, 0x15,
0x23, 0xb0, 0x0e, 0x03, 0xc9, 0x9f, 0x01, 0xf2, 0xe7, 0x98, 0x2d, 0x29, 0x8b, 0x1c, 0x62, 0x2e,
0x7a, 0x88, 0xf2, 0x3f, 0x48, 0xb0, 0x10, 0x9c, 0x6c, 0xda, 0xf0, 0xf8, 0x11, 0x54, 0xd9, 0x1d,
0x48, 0x97, 0x98, 0x27, 0xef, 0x46, 0x5c, 0x18, 0xbb, 0x7b, 0x0a, 0xf8, 0x2f, 0xc8, 0x89, 0x12,
0x1c, 0x59, 0xf6, 0xa1, 0x6e, 0xf6, 0xbb, 0x44, 0x32, 0xcf, 0x28, 0x6a, 0x1c, 0xf8, 0x98, 0xc0,
0xe4, 0xaf, 0x25, 0xb8, 0xf8, 0x74, 0xa8, 0xa9, 0x2e, 0x0e, 0xe4, 0x09, 0xb3, 0x3e, 0x4a, 0xfb,
0xc0, 0x7b, 0x15, 0x96, 0xcb, 0xd6, 0xc7, 0x67, 0xd8, 0xf2, 0x16, 0x9c, 0x55, 0xb0, 0x83, 0x4d,
0x2d, 0xf4, 0x71, 0xea, 0x1e, 0xc4, 0x10, 0xda, 0x49, 0xec, 0x66, 0x39, 0x7b, 0x96, 0xb0, 0x75,
0x6d, 0xc2, 0xd6, 0xe5, 0xfe, 0x87, 0xe4, 0x09, 0x74, 0x1e, 0x57, 0xfe, 0x1f, 0x09, 0x16, 0xee,
0x6b, 0xde, 0x7c, 0xaf, 0x2d, 0x2f, 0x8c, 0xe6, 0x4d, 0xf9, 0x78, 0xde, 0xf4, 0xaa, 0x1c, 0x09,
0x77, 0xa9, 0xe6, 0xc8, 0xf0, 0x42, 0x85, 0x4d, 0x1f, 0x1d, 0xc8, 0xfb, 0xe2, 0x9a, 0x59, 0xc1,
0xfb, 0xd8, 0xc6, 0x66, 0x0f, 0x6f, 0x5a, 0xbd, 0xc3, 0xc0, 0xd3, 0x32, 0x29, 0xf8, 0xb4, 0x6c,
0xda, 0xa7, 0x6a, 0x37, 0xff, 0x5a, 0x82, 0x85, 0x58, 0x3f, 0x0b, 0x35, 0x00, 0x9e, 0x9a, 0x3d,
0xde, 0xe8, 0x6b, 0x9e, 0x42, 0x35, 0x28, 0x7b, 0x6d, 0xbf, 0xa6, 0x84, 0xaa, 0x30, 0xb7, 0x6b,
0x51, 0xec, 0x66, 0x0e, 0x35, 0xa1, 0xc6, 0x08, 0x47, 0xbd, 0x1e, 0x76, 0x9c, 0x66, 0x5e, 0x40,
0xd6, 0x55, 0x7d, 0x30, 0xb2, 0x71, 0xb3, 0x80, 0xea, 0x50, 0xd9, 0xb5, 0xf8, 0xc3, 0xbc, 0x66,
0x11, 0x21, 0x68, 0x78, 0xaf, 0xf4, 0x38, 0x51, 0x29, 0x00, 0xf3, 0xc8, 0xe6, 0x6e, 0x3e, 0x0b,
0xf6, 0x44, 0x76, 0x8f, 0x87, 0x18, 0x2d, 0xc1, 0xe9, 0xa7, 0xa6, 0x86, 0xf7, 0x75, 0x13, 0x6b,
0xfe, 0xa7, 0xe6, 0x29, 0x74, 0x1a, 0xe6, 0xb7, 0xb0, 0xdd, 0xc7, 0x01, 0x60, 0x0e, 0x2d, 0x40,
0x7d, 0x4b, 0x7f, 0x19, 0x00, 0xe5, 0xe5, 0x42, 0x59, 0x6a, 0x4a, 0x2b, 0xff, 0xbb, 0x04, 0x15,
0x52, 0xcf, 0x3c, 0xb0, 0x2c, 0x5b, 0x43, 0x43, 0x40, 0xf4, 0x1d, 0xab, 0x31, 0xb4, 0x4c, 0xf1,
0x3a, 0x1c, 0xbd, 0x97, 0x92, 0x33, 0xc5, 0x51, 0xb9, 0x1e, 0xb6, 0xaf, 0xa7, 0x50, 0x44, 0xd0,
0xe5, 0x53, 0xc8, 0xa0, 0x33, 0xee, 0xea, 0x06, 0xde, 0xd5, 0x7b, 0x87, 0xde, 0x03, 0x97, 0x31,
0x33, 0x46, 0x50, 0xbd, 0x19, 0x23, 0xd5, 0x33, 0x1f, 0xb0, 0xc7, 0xc6, 0x9e, 0x21, 0xca, 0xa7,
0xd0, 0x73, 0x38, 0xb3, 0x81, 0x03, 0x8e, 0xc7, 0x9b, 0x70, 0x25, 0x7d, 0xc2, 0x18, 0xf2, 0x09,
0xa7, 0xdc, 0x84, 0x22, 0x6d, 0x16, 0xa3, 0x24, 0xdf, 0x14, 0xfc, 0x31, 0x57, 0xfb, 0x72, 0x3a,
0x82, 0xe0, 0xf6, 0x19, 0xcc, 0x47, 0x7e, 0x02, 0x82, 0x6e, 0x24, 0x90, 0x25, 0xff, 0x98, 0xa7,
0x7d, 0x33, 0x0b, 0xaa, 0x98, 0xab, 0x0f, 0x8d, 0xf0, 0x1b, 0x58, 0x94, 0xd4, 0x14, 0x48, 0x7c,
0xbd, 0xdf, 0xbe, 0x91, 0x01, 0x53, 0x4c, 0x64, 0x40, 0x33, 0xfa, 0x93, 0x04, 0x74, 0x73, 0x2c,
0x83, 0xb0, 0xba, 0xfd, 0x2c, 0x13, 0xae, 0x98, 0xee, 0x98, 0x2a, 0x41, 0xec, 0x95, 0x3b, 0xba,
0x9d, 0xcc, 0x26, 0xed, 0xf9, 0x7d, 0xfb, 0x4e, 0x66, 0x7c, 0x31, 0xf5, 0xef, 0xb3, 0x4b, 0xaa,
0xa4, 0x97, 0xe2, 0xe8, 0xfd, 0x64, 0x76, 0x63, 0x9e, 0xb8, 0xb7, 0x57, 0x4e, 0x42, 0x22, 0x84,
0xf8, 0x82, 0xde, 0x2e, 0x25, 0xbc, 0xb5, 0x8e, 0xda, 0x9d, 0xc7, 0x2f, 0xfd, 0x19, 0x79, 0xfb,
0xfd, 0x13, 0x50, 0x08, 0x01, 0xac, 0xe8, 0x6f, 0x3e, 0x3c, 0x33, 0xbc, 0x33, 0x51, 0x6b, 0xa6,
0xb3, 0xc1, 0x4f, 0x61, 0x3e, 0xf2, 0x4e, 0x28, 0xd1, 0x6a, 0x92, 0xdf, 0x12, 0xb5, 0xc7, 0xc5,
0x6b, 0x66, 0x92, 0x91, 0xcb, 0x3a, 0x94, 0xa2, 0xfd, 0x09, 0x17, 0x7a, 0xed, 0x9b, 0x59, 0x50,
0xc5, 0x42, 0x1c, 0xea, 0x2e, 0x23, 0x17, 0x5e, 0xe8, 0x56, 0x32, 0x8f, 0xe4, 0xcb, 0xba, 0xf6,
0xbb, 0x19, 0xb1, 0xc5, 0xa4, 0xbf, 0x07, 0x68, 0xe7, 0x80, 0xd4, 0x30, 0xe6, 0xbe, 0xde, 0x1f,
0xd9, 0x2a, 0x7b, 0x0c, 0x94, 0xe6, 0xa3, 0xe3, 0xa8, 0x29, 0xba, 0x32, 0x96, 0x42, 0x4c, 0xde,
0x05, 0xd8, 0xc0, 0xee, 0x16, 0x76, 0x6d, 0xa2, 0xa0, 0xd7, 0x13, 0xcf, 0xdb, 0x47, 0xf0, 0xa6,
0x7a, 0x67, 0x22, 0x5e, 0x20, 0x24, 0x34, 0xb7, 0x54, 0x93, 0x94, 0xef, 0xfe, 0x1b, 0xba, 0x5b,
0x89, 0xe4, 0x51, 0xb4, 0x94, 0x0d, 0x4d, 0xc5, 0x16, 0x53, 0x1e, 0x89, 0x30, 0x1b, 0xe8, 0xa6,
0x46, 0xdd, 0x8f, 0x2f, 0x73, 0xf2, 0x55, 0x59, 0xd4, 0xfd, 0x8c, 0xc1, 0x17, 0x13, 0x7f, 0x29,
0xd1, 0x1f, 0x14, 0x45, 0x10, 0x9e, 0xe9, 0xee, 0xc1, 0xf6, 0x40, 0x35, 0x9d, 0x2c, 0x22, 0x50,
0xc4, 0x13, 0x88, 0xc0, 0xf1, 0x85, 0x08, 0x1a, 0xd4, 0x43, 0x3d, 0x52, 0x94, 0xf4, 0x1a, 0x2d,
0xa9, 0xe1, 0xdb, 0x5e, 0x9e, 0x8c, 0x28, 0x66, 0x39, 0x80, 0xba, 0xa7, 0xd2, 0x6c, 0x73, 0x6f,
0xa4, 0x49, 0xea, 0xe3, 0xa4, 0x58, 0x64, 0x32, 0x6a, 0xd0, 0x22, 0xe3, 0x2d, 0x20, 0x94, 0xad,
0x75, 0x38, 0xce, 0x22, 0xd3, 0xfb, 0x4a, 0xcc, 0xe5, 0x44, 0xda, 0xad, 0xc9, 0xfe, 0x2c, 0xb1,
0x7b, 0x9c, 0xe8, 0x72, 0x52, 0xba, 0xb7, 0xf2, 0x29, 0xf4, 0x0c, 0x4a, 0xfc, 0x97, 0xc4, 0x6f,
0x8f, 0x2f, 0x08, 0x39, 0xf7, 0x6b, 0x13, 0xb0, 0x04, 0xe3, 0x43, 0x58, 0x4a, 0x29, 0x07, 0x13,
0x43, 0xe1, 0xf8, 0xd2, 0x71, 0x92, 0x93, 0x56, 0x01, 0xc5, 0x7f, 0xae, 0x93, 0x78, 0x4c, 0xa9,
0xbf, 0xea, 0xc9, 0x30, 0x45, 0xfc, 0x17, 0x37, 0x89, 0x53, 0xa4, 0xfe, 0x30, 0x67, 0xd2, 0x14,
0x4f, 0x00, 0xfc, 0xa2, 0x2f, 0xf1, 0x3c, 0x62, 0x35, 0xe1, 0x04, 0x96, 0x2b, 0xdf, 0x55, 0xa0,
0xec, 0xbd, 0xfd, 0x7a, 0x03, 0xf9, 0xff, 0x1b, 0x48, 0xc8, 0x3f, 0x85, 0xf9, 0xc8, 0xef, 0x4c,
0x12, 0x8d, 0x27, 0xf9, 0xb7, 0x28, 0x93, 0x4e, 0xe8, 0x19, 0xff, 0x2f, 0x08, 0x22, 0x36, 0xbf,
0x93, 0x96, 0xd4, 0x47, 0xc3, 0xf2, 0x04, 0xc6, 0xbf, 0xde, 0x41, 0xf8, 0x31, 0x40, 0x20, 0xfc,
0x8e, 0xbf, 0x27, 0x27, 0x11, 0x65, 0xd2, 0x6e, 0x19, 0x89, 0x11, 0xf6, 0x46, 0x96, 0x3b, 0xcd,
0x74, 0x1f, 0x99, 0x1e, 0x57, 0x9f, 0x42, 0x2d, 0xf8, 0x04, 0x05, 0x25, 0xfe, 0xe6, 0x3e, 0xfe,
0x46, 0x65, 0xd2, 0x2a, 0xb6, 0x4e, 0xe8, 0x7a, 0x27, 0xb0, 0x73, 0x88, 0x83, 0x8a, 0x76, 0xa9,
0x52, 0x1c, 0x54, 0x4a, 0x6f, 0x2c, 0x31, 0x54, 0xa5, 0xb7, 0xbe, 0x5e, 0x8b, 0xcb, 0x5a, 0xbd,
0xfb, 0x3b, 0xef, 0xf7, 0x75, 0xf7, 0x60, 0xb4, 0x47, 0xbe, 0xdc, 0x61, 0xa8, 0xef, 0xea, 0x16,
0xff, 0xeb, 0x8e, 0xa7, 0xd2, 0x77, 0x28, 0xf5, 0x1d, 0x32, 0xc7, 0x70, 0x6f, 0xaf, 0x44, 0x47,
0x77, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x22, 0xe3, 0x1e, 0xab, 0x45, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -5680,6 +5747,7 @@ type DataNodeClient interface {
GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error)
Compaction(ctx context.Context, in *CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error)
GetCompactionState(ctx context.Context, in *CompactionStateRequest, opts ...grpc.CallOption) (*CompactionStateResponse, error)
SyncSegments(ctx context.Context, in *SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
ResendSegmentStats(ctx context.Context, in *ResendSegmentStatsRequest, opts ...grpc.CallOption) (*ResendSegmentStatsResponse, error)
@ -5766,6 +5834,15 @@ func (c *dataNodeClient) GetCompactionState(ctx context.Context, in *CompactionS
return out, nil
}
func (c *dataNodeClient) SyncSegments(ctx context.Context, in *SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
out := new(commonpb.Status)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/SyncSegments", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataNodeClient) Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
out := new(commonpb.Status)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/Import", in, out, opts...)
@ -5804,6 +5881,7 @@ type DataNodeServer interface {
GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
Compaction(context.Context, *CompactionPlan) (*commonpb.Status, error)
GetCompactionState(context.Context, *CompactionStateRequest) (*CompactionStateResponse, error)
SyncSegments(context.Context, *SyncSegmentsRequest) (*commonpb.Status, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(context.Context, *ImportTaskRequest) (*commonpb.Status, error)
ResendSegmentStats(context.Context, *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error)
@ -5838,6 +5916,9 @@ func (*UnimplementedDataNodeServer) Compaction(ctx context.Context, req *Compact
func (*UnimplementedDataNodeServer) GetCompactionState(ctx context.Context, req *CompactionStateRequest) (*CompactionStateResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetCompactionState not implemented")
}
func (*UnimplementedDataNodeServer) SyncSegments(ctx context.Context, req *SyncSegmentsRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method SyncSegments not implemented")
}
func (*UnimplementedDataNodeServer) Import(ctx context.Context, req *ImportTaskRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method Import not implemented")
}
@ -5996,6 +6077,24 @@ func _DataNode_GetCompactionState_Handler(srv interface{}, ctx context.Context,
return interceptor(ctx, in, info, handler)
}
func _DataNode_SyncSegments_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(SyncSegmentsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataNodeServer).SyncSegments(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.data.DataNode/SyncSegments",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataNodeServer).SyncSegments(ctx, req.(*SyncSegmentsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _DataNode_Import_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ImportTaskRequest)
if err := dec(in); err != nil {
@ -6086,6 +6185,10 @@ var _DataNode_serviceDesc = grpc.ServiceDesc{
MethodName: "GetCompactionState",
Handler: _DataNode_GetCompactionState_Handler,
},
{
MethodName: "SyncSegments",
Handler: _DataNode_SyncSegments_Handler,
},
{
MethodName: "Import",
Handler: _DataNode_Import_Handler,

View File

@ -80,6 +80,8 @@ type DataNode interface {
Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error)
// GetCompactionState get states of all compation tasks
GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error)
// SyncSegments is called by DataCoord, to sync the segments meta when complete compaction
SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error)
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
//

View File

@ -76,3 +76,7 @@ func (m *GrpcDataNodeClient) ResendSegmentStats(ctx context.Context, req *datapb
func (m *GrpcDataNodeClient) AddSegment(ctx context.Context, in *datapb.AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcDataNodeClient) SyncSegments(ctx context.Context, in *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}