From 432b5ddf2eedadb4f640ef2d967ae937590c4f89 Mon Sep 17 00:00:00 2001 From: XuanYang-cn Date: Fri, 23 Sep 2022 10:22:52 +0800 Subject: [PATCH] Add SyncSegments into proto (#19312) See also: #19072 Signed-off-by: yangxuan Signed-off-by: yangxuan --- internal/datacoord/mock_test.go | 4 + internal/datanode/data_node.go | 59 ++ internal/datanode/data_node_test.go | 84 +++ internal/datanode/flow_graph_delete_node.go | 15 +- .../datanode/flow_graph_delete_node_test.go | 53 +- internal/datanode/flow_graph_manager.go | 26 +- internal/datanode/flow_graph_manager_test.go | 41 ++ internal/datanode/segment_replica.go | 55 +- internal/datanode/segment_replica_test.go | 137 +++- .../distributed/datanode/client/client.go | 14 + internal/distributed/datanode/service.go | 4 + internal/distributed/datanode/service_test.go | 4 + internal/mocks/mock_datanode.go | 47 ++ internal/proto/data_coord.proto | 14 +- internal/proto/datapb/data_coord.pb.go | 633 ++++++++++-------- internal/types/types.go | 2 + internal/util/mock/grpc_datanode_client.go | 4 + 17 files changed, 872 insertions(+), 324 deletions(-) diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index 82b3df5f20..838de9f084 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -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 diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index a40f09d6fe..b9af916962 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -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", diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 75d0ba1539..30bc4aa90f 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -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) { diff --git a/internal/datanode/flow_graph_delete_node.go b/internal/datanode/flow_graph_delete_node.go index b2ec7a93bb..713d9fd248 100644 --- a/internal/datanode/flow_graph_delete_node.go +++ b/internal/datanode/flow_graph_delete_node.go @@ -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", diff --git a/internal/datanode/flow_graph_delete_node_test.go b/internal/datanode/flow_graph_delete_node_test.go index d06ba2cd4e..68cb3d5780 100644 --- a/internal/datanode/flow_graph_delete_node_test.go +++ b/internal/datanode/flow_graph_delete_node_test.go @@ -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) }) } } diff --git a/internal/datanode/flow_graph_manager.go b/internal/datanode/flow_graph_manager.go index 3d140d25c5..ee1b0231e8 100644 --- a/internal/datanode/flow_graph_manager.go +++ b/internal/datanode/flow_graph_manager.go @@ -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 { diff --git a/internal/datanode/flow_graph_manager_test.go b/internal/datanode/flow_graph_manager_test.go index 99c773a20a..c981a0668f 100644 --- a/internal/datanode/flow_graph_manager_test.go +++ b/internal/datanode/flow_graph_manager_test.go @@ -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{ diff --git a/internal/datanode/segment_replica.go b/internal/datanode/segment_replica.go index 97343800cd..cd45fd6915 100644 --- a/internal/datanode/segment_replica.go +++ b/internal/datanode/segment_replica.go @@ -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 } diff --git a/internal/datanode/segment_replica_test.go b/internal/datanode/segment_replica_test.go index 4447dc4f2c..ec4e4ee639 100644 --- a/internal/datanode/segment_replica_test.go +++ b/internal/datanode/segment_replica_test.go @@ -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)) + } + + }) + } }) } diff --git a/internal/distributed/datanode/client/client.go b/internal/distributed/datanode/client/client.go index 2ea6fcb280..8a9c4778fe 100644 --- a/internal/distributed/datanode/client/client.go +++ b/internal/distributed/datanode/client/client.go @@ -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 +} diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index 54700a7d72..512ada37db 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -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) +} diff --git a/internal/distributed/datanode/service_test.go b/internal/distributed/datanode/service_test.go index ca8a7e224f..abd2c937f2 100644 --- a/internal/distributed/datanode/service_test.go +++ b/internal/distributed/datanode/service_test.go @@ -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 diff --git a/internal/mocks/mock_datanode.go b/internal/mocks/mock_datanode.go index 8bac89eade..fc733aea19 100644 --- a/internal/mocks/mock_datanode.go +++ b/internal/mocks/mock_datanode.go @@ -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) diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index be11543c9b..36cd8243ad 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -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; } diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index 63bd292dba..5010e74714 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -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, diff --git a/internal/types/types.go b/internal/types/types.go index 8af8d5fa91..af30e33029 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -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 // diff --git a/internal/util/mock/grpc_datanode_client.go b/internal/util/mock/grpc_datanode_client.go index ece1f4017b..9970095bb3 100644 --- a/internal/util/mock/grpc_datanode_client.go +++ b/internal/util/mock/grpc_datanode_client.go @@ -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 +}