Implement RC, DC, DN calling path for import. (#16321)

/kind feature

issue: #15604
Signed-off-by: Yuchen Gao <yuchen.gao@zilliz.com>
pull/16317/head
Ten Thousand Leaves 2022-04-01 11:33:28 +08:00 committed by GitHub
parent 376a7f6567
commit 45be3deb3f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 542 additions and 296 deletions

View File

@ -19,12 +19,11 @@ package datacoord
import (
"context"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"go.uber.org/zap"
)
// Cluster provides interfaces to interact with datanode cluster
@ -137,6 +136,11 @@ func (c *Cluster) Flush(ctx context.Context, segments []*datapb.SegmentInfo, mar
}
}
// Import sends import requests to DataNodes whose ID==nodeID.
func (c *Cluster) Import(ctx context.Context, nodeID int64, it *datapb.ImportTaskRequest) {
c.sessionManager.Import(ctx, nodeID, it)
}
// GetSessions returns all sessions
func (c *Cluster) GetSessions() []*Session {
return c.sessionManager.GetSessions()

View File

@ -20,6 +20,7 @@ import (
"context"
"errors"
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
@ -544,3 +545,35 @@ func TestCluster_Flush(t *testing.T) {
})
//TODO add a method to verify datanode has flush request after client injection is available
}
func TestCluster_Import(t *testing.T) {
kv := getMetaKv(t)
defer func() {
kv.RemoveWithPrefix("")
kv.Close()
}()
ctx, cancel := context.WithTimeout(context.TODO(), 100*time.Millisecond)
defer cancel()
sessionManager := NewSessionManager()
channelManager, err := NewChannelManager(kv, newMockHandler())
assert.Nil(t, err)
cluster := NewCluster(sessionManager, channelManager)
defer cluster.Close()
addr := "localhost:8080"
info := &NodeInfo{
Address: addr,
NodeID: 1,
}
nodes := []*NodeInfo{info}
err = cluster.Startup(ctx, nodes)
assert.Nil(t, err)
err = cluster.Watch("chan-1", 1)
assert.NoError(t, err)
assert.NotPanics(t, func() {
cluster.Import(ctx, 1, &datapb.ImportTaskRequest{})
})
time.Sleep(500 * time.Millisecond)
}

View File

@ -196,7 +196,7 @@ func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.Compact
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: "not implemented"}, nil
}
func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTask) (*commonpb.Status, error) {
func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRequest) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}

View File

@ -2266,9 +2266,16 @@ func TestImport(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
resp, err := svr.Import(svr.ctx, &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
err := svr.channelManager.AddNode(0)
assert.Nil(t, err)
err = svr.channelManager.Watch(&channel{"ch1", 0})
assert.Nil(t, err)
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
},
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.GetErrorCode())
@ -2276,13 +2283,56 @@ func TestImport(t *testing.T) {
etcd.StopEtcdServer()
})
t.Run("no free node", func(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
err := svr.channelManager.AddNode(0)
assert.Nil(t, err)
err = svr.channelManager.Watch(&channel{"ch1", 0})
assert.Nil(t, err)
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
},
WorkingNodes: []int64{0},
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.GetErrorCode())
etcd.StopEtcdServer()
})
t.Run("no datanode available", func(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
},
})
assert.Nil(t, err)
assert.EqualValues(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())
etcd.StopEtcdServer()
})
t.Run("with closed server", func(t *testing.T) {
svr := newTestServer(t, nil)
closeTestServer(t, svr)
resp, err := svr.Import(svr.ctx, &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
},
})
assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())

View File

@ -19,6 +19,7 @@ package datacoord
import (
"context"
"fmt"
"math/rand"
"strconv"
"sync/atomic"
"time"
@ -960,8 +961,8 @@ func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateR
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (s *Server) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
log.Info("receive import request")
func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
log.Info("receive import request", zap.Any("import task request", itr))
resp := &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -969,11 +970,49 @@ func (s *Server) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.Im
}
if s.isClosed() {
log.Warn("failed to import because of closed server", zap.Int64("collection ID", req.GetCollectionId()))
log.Warn("failed to import because of closed server", zap.Any("import task request", itr))
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.NodeID)
return resp, nil
}
workingNodes := itr.WorkingNodes
nodes := s.channelManager.store.GetNodes()
if len(nodes) == 0 {
log.Error("import failed as all dataNodes are offline", zap.Any("import task request", itr))
return resp, nil
}
avaNodes := getDiff(nodes, workingNodes)
if len(avaNodes) > 0 {
// If there exists available DataNodes, pick one at random.
dnID := avaNodes[rand.Intn(len(avaNodes))]
log.Info("picking a free dataNode",
zap.Any("all dataNodes", nodes),
zap.Int64("picking free dataNode with ID", dnID))
s.cluster.Import(ctx, dnID, itr)
} else {
// No DataNodes are available, choose a still working DataNode randomly.
dnID := nodes[rand.Intn(len(nodes))]
log.Info("all dataNodes are busy, picking a random dataNode still",
zap.Any("all dataNodes", nodes),
zap.Int64("picking dataNode with ID", dnID))
s.cluster.Import(ctx, dnID, itr)
}
resp.Status.ErrorCode = commonpb.ErrorCode_Success
return resp, nil
}
// getDiff returns the difference of base and remove. i.e. all items that are in `base` but not in `remove`.
func getDiff(base, remove []int64) []int64 {
mb := make(map[int64]struct{}, len(remove))
for _, x := range remove {
mb[x] = struct{}{}
}
var diff []int64
for _, x := range base {
if _, found := mb[x]; !found {
diff = append(diff, x)
}
}
return diff
}

View File

@ -31,6 +31,8 @@ import (
const (
flushTimeout = 5 * time.Second
// TODO: evaluate and update import timeout.
importTimeout = 3 * time.Hour
)
// SessionManager provides the grpc interfaces of cluster
@ -148,6 +150,29 @@ func (c *SessionManager) execCompaction(nodeID int64, plan *datapb.CompactionPla
log.Info("success to execute compaction", zap.Int64("node", nodeID), zap.Any("planID", plan.GetPlanID()))
}
// Import is a grpc interface. It will send request to DataNode with provided `nodeID` asynchronously.
func (c *SessionManager) Import(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest) {
go c.execImport(ctx, nodeID, itr)
}
// execImport gets the corresponding DataNode with its ID and calls its Import method.
func (c *SessionManager) execImport(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest) {
cli, err := c.getClient(ctx, nodeID)
if err != nil {
log.Warn("failed to get client for import", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
ctx, cancel := context.WithTimeout(ctx, importTimeout)
defer cancel()
resp, err := cli.Import(ctx, itr)
if err := VerifyResponse(resp, err); err != nil {
log.Warn("failed to import", zap.Int64("node", nodeID), zap.Error(err))
return
}
log.Info("success to import", zap.Int64("node", nodeID), zap.Any("import task", itr))
}
func (c *SessionManager) getClient(ctx context.Context, nodeID int64) (types.DataNode, error) {
c.sessions.RLock()
session, ok := c.sessions.data[nodeID]

View File

@ -776,14 +776,14 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan
}
// 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.ImportTask) (*commonpb.Status, error) {
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
log.Info("receive import request")
if !node.isHealthy() {
log.Warn("DataNode.Import failed",
zap.Int64("collection ID", req.GetCollectionId()),
zap.Int64("partition ID", req.GetPartitionId()),
zap.Int64("taskID", req.GetTaskId()),
zap.Int64("collection ID", req.GetImportTask().GetCollectionId()),
zap.Int64("partition ID", req.GetImportTask().GetPartitionId()),
zap.Int64("taskID", req.GetImportTask().GetTaskId()),
zap.Error(errDataNodeIsUnhealthy(Params.DataNodeCfg.NodeID)))
return &commonpb.Status{

View File

@ -317,9 +317,11 @@ func TestDataNode(t *testing.T) {
})
t.Run("Test Import", func(t *testing.T) {
req := &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
req := &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
PartitionId: 100,
},
}
stat, err := node.Import(node.ctx, req)
assert.NoError(t, err)

View File

@ -488,7 +488,7 @@ func (c *Client) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (c *Client) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -326,6 +326,6 @@ func (s *Server) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (s *Server) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
func (s *Server) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
return s.dataCoord.Import(ctx, req)
}

View File

@ -165,7 +165,7 @@ func (m *MockDataCoord) DropVirtualChannel(ctx context.Context, req *datapb.Drop
return m.dropVChanResp, m.err
}
func (m *MockDataCoord) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
func (m *MockDataCoord) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
return m.importResp, m.err
}

View File

@ -183,7 +183,7 @@ func (c *Client) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*c
}
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (c *Client) Import(ctx context.Context, req *datapb.ImportTask) (*commonpb.Status, error) {
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -354,6 +354,6 @@ func (s *Server) Compaction(ctx context.Context, request *datapb.CompactionPlan)
return s.datanode.Compaction(ctx, request)
}
func (s *Server) Import(ctx context.Context, request *datapb.ImportTask) (*commonpb.Status, error) {
func (s *Server) Import(ctx context.Context, request *datapb.ImportTaskRequest) (*commonpb.Status, error) {
return s.datanode.Import(ctx, request)
}

View File

@ -111,7 +111,7 @@ func (m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPla
func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockDataNode) Import(ctx context.Context, req *datapb.ImportTask) (*commonpb.Status, error) {
func (m *MockDataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
return m.status, m.err
}

View File

@ -466,7 +466,7 @@ func (m *MockDataCoord) DropVirtualChannel(ctx context.Context, req *datapb.Drop
return &datapb.DropVirtualChannelResponse{}, nil
}
func (m *MockDataCoord) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
func (m *MockDataCoord) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
return nil, nil
}

View File

@ -210,7 +210,7 @@ func TestGrpcService(t *testing.T) {
core.CallReleasePartitionService = func(ctx context.Context, ts typeutil.Timestamp, dbID, collectionID typeutil.UniqueID, partitionIDs []typeutil.UniqueID) error {
return nil
}
core.CallImportService = func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
core.CallImportService = func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return nil
}

View File

@ -43,7 +43,7 @@ service DataCoord {
rpc DropVirtualChannel(DropVirtualChannelRequest) returns (DropVirtualChannelResponse) {}
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
rpc Import(ImportTask) returns (ImportTaskResponse) {}
rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {}
}
service DataNode {
@ -58,7 +58,7 @@ service DataNode {
rpc Compaction(CompactionPlan) returns (common.Status) {}
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
rpc Import(ImportTask) returns(common.Status) {}
rpc Import(ImportTaskRequest) returns(common.Status) {}
}
message FlushRequest {
@ -441,15 +441,21 @@ message ImportTaskInfo {
int64 request_id = 2; // Request ID of the import task.
int64 datanode_id = 3; // ID of DataNode that processes the task.
int64 collection_id = 4; // Collection ID for the import task.
int64 partition_id = 5; // Partition ID for the import task.
int64 partition_id = 5; // Partition ID for the import task.
string bucket = 6; // Bucket for the import task.
bool row_based = 7; // Boolean indicating whether import files are row-based or column-based.
repeated string files = 8; // A list of files to import.
int64 create_ts = 9; // Timestamp when the import task is created.
ImportTaskState state = 10; // State of the import task.
ImportTaskState state = 10; // State of the import task.
}
message ImportTaskResponse {
common.Status status = 1;
int64 datanode_id = 2; // which datanode takes this task
}
int64 datanode_id = 2; // which datanode takes this task
}
message ImportTaskRequest {
common.MsgBase base = 1;
ImportTask import_task = 2; // Target import task.
repeated int64 working_nodes = 3; // DataNodes that are currently working.
}

View File

@ -3429,6 +3429,61 @@ func (m *ImportTaskResponse) GetDatanodeId() int64 {
return 0
}
type ImportTaskRequest struct {
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
ImportTask *ImportTask `protobuf:"bytes,2,opt,name=import_task,json=importTask,proto3" json:"import_task,omitempty"`
WorkingNodes []int64 `protobuf:"varint,3,rep,packed,name=working_nodes,json=workingNodes,proto3" json:"working_nodes,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
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{54}
}
func (m *ImportTaskRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ImportTaskRequest.Unmarshal(m, b)
}
func (m *ImportTaskRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_ImportTaskRequest.Marshal(b, m, deterministic)
}
func (m *ImportTaskRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_ImportTaskRequest.Merge(m, src)
}
func (m *ImportTaskRequest) XXX_Size() int {
return xxx_messageInfo_ImportTaskRequest.Size(m)
}
func (m *ImportTaskRequest) XXX_DiscardUnknown() {
xxx_messageInfo_ImportTaskRequest.DiscardUnknown(m)
}
var xxx_messageInfo_ImportTaskRequest proto.InternalMessageInfo
func (m *ImportTaskRequest) GetBase() *commonpb.MsgBase {
if m != nil {
return m.Base
}
return nil
}
func (m *ImportTaskRequest) GetImportTask() *ImportTask {
if m != nil {
return m.ImportTask
}
return nil
}
func (m *ImportTaskRequest) GetWorkingNodes() []int64 {
if m != nil {
return m.WorkingNodes
}
return nil
}
func init() {
proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value)
proto.RegisterEnum("milvus.proto.data.CompactionType", CompactionType_name, CompactionType_value)
@ -3486,214 +3541,218 @@ func init() {
proto.RegisterType((*ImportTaskState)(nil), "milvus.proto.data.ImportTaskState")
proto.RegisterType((*ImportTaskInfo)(nil), "milvus.proto.data.ImportTaskInfo")
proto.RegisterType((*ImportTaskResponse)(nil), "milvus.proto.data.ImportTaskResponse")
proto.RegisterType((*ImportTaskRequest)(nil), "milvus.proto.data.ImportTaskRequest")
}
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 3222 bytes of a gzipped FileDescriptorProto
// 3270 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3b, 0x5b, 0x6f, 0x1b, 0xc7,
0xd5, 0x5e, 0xde, 0x79, 0x78, 0x11, 0x3d, 0x76, 0x64, 0x9a, 0xb6, 0x65, 0x79, 0x1d, 0x3b, 0x8a,
0xe3, 0xd8, 0x89, 0xfc, 0x05, 0x09, 0xbe, 0xdc, 0x10, 0x49, 0x91, 0x42, 0x7c, 0x92, 0x3f, 0x65,
0xa5, 0x24, 0x1f, 0xbe, 0x14, 0x25, 0x56, 0xdc, 0x11, 0xb5, 0xd1, 0x5e, 0x98, 0x9d, 0xa5, 0x65,
0xe7, 0x25, 0x46, 0x03, 0x14, 0x68, 0x11, 0xa4, 0x2d, 0xfa, 0x54, 0xa0, 0x28, 0x8a, 0x3e, 0xb5,
0xe8, 0x4b, 0xfb, 0xd8, 0x16, 0x7d, 0x0f, 0xd2, 0x7f, 0xd1, 0x97, 0xbe, 0xf5, 0x37, 0x14, 0x73,
0xd9, 0xd9, 0x0b, 0x97, 0xe4, 0x4a, 0xb2, 0xe3, 0x37, 0xcd, 0xd9, 0x73, 0xce, 0xcc, 0x9c, 0xfb,
0x39, 0x1c, 0x41, 0xcb, 0xd0, 0x7d, 0xbd, 0xd7, 0x77, 0x5d, 0xcf, 0xb8, 0x33, 0xf4, 0x5c, 0xdf,
0x45, 0x67, 0x6d, 0xd3, 0x7a, 0x30, 0x22, 0x7c, 0x75, 0x87, 0x7e, 0xee, 0xd4, 0xfb, 0xae, 0x6d,
0xbb, 0x0e, 0x07, 0x75, 0x9a, 0xa6, 0xe3, 0x63, 0xcf, 0xd1, 0x2d, 0xb1, 0xae, 0x47, 0x09, 0x3a,
0x75, 0xd2, 0x3f, 0xc0, 0xb6, 0xce, 0x57, 0xea, 0x43, 0xa8, 0xaf, 0x5b, 0x23, 0x72, 0xa0, 0xe1,
0xcf, 0x47, 0x98, 0xf8, 0xe8, 0x15, 0x28, 0xec, 0xe9, 0x04, 0xb7, 0x95, 0x45, 0x65, 0xa9, 0xb6,
0x7c, 0xf9, 0x4e, 0x6c, 0x2f, 0xb1, 0xcb, 0x16, 0x19, 0xac, 0xe8, 0x04, 0x6b, 0x0c, 0x13, 0x21,
0x28, 0x18, 0x7b, 0xdd, 0xb5, 0x76, 0x6e, 0x51, 0x59, 0xca, 0x6b, 0xec, 0x6f, 0xa4, 0x42, 0xbd,
0xef, 0x5a, 0x16, 0xee, 0xfb, 0xa6, 0xeb, 0x74, 0xd7, 0xda, 0x05, 0xf6, 0x2d, 0x06, 0x53, 0x7f,
0xad, 0x40, 0x43, 0x6c, 0x4d, 0x86, 0xae, 0x43, 0x30, 0xba, 0x07, 0x25, 0xe2, 0xeb, 0xfe, 0x88,
0x88, 0xdd, 0x2f, 0xa5, 0xee, 0xbe, 0xc3, 0x50, 0x34, 0x81, 0x9a, 0x69, 0xfb, 0xfc, 0xf8, 0xf6,
0x68, 0x01, 0x80, 0xe0, 0x81, 0x8d, 0x1d, 0xbf, 0xbb, 0x46, 0xda, 0x85, 0xc5, 0xfc, 0x52, 0x5e,
0x8b, 0x40, 0xd4, 0x5f, 0x28, 0xd0, 0xda, 0x09, 0x96, 0x81, 0x74, 0xce, 0x43, 0xb1, 0xef, 0x8e,
0x1c, 0x9f, 0x1d, 0xb0, 0xa1, 0xf1, 0x05, 0xba, 0x06, 0xf5, 0xfe, 0x81, 0xee, 0x38, 0xd8, 0xea,
0x39, 0xba, 0x8d, 0xd9, 0x51, 0xaa, 0x5a, 0x4d, 0xc0, 0xee, 0xeb, 0x36, 0xce, 0x74, 0xa2, 0x45,
0xa8, 0x0d, 0x75, 0xcf, 0x37, 0x63, 0x32, 0x8b, 0x82, 0xd4, 0xdf, 0x2a, 0x30, 0xff, 0x1e, 0x21,
0xe6, 0xc0, 0x19, 0x3b, 0xd9, 0x3c, 0x94, 0x1c, 0xd7, 0xc0, 0xdd, 0x35, 0x76, 0xb4, 0xbc, 0x26,
0x56, 0xe8, 0x12, 0x54, 0x87, 0x18, 0x7b, 0x3d, 0xcf, 0xb5, 0x82, 0x83, 0x55, 0x28, 0x40, 0x73,
0x2d, 0x8c, 0x3e, 0x84, 0xb3, 0x24, 0xc1, 0x88, 0xb4, 0xf3, 0x8b, 0xf9, 0xa5, 0xda, 0xf2, 0xf5,
0x3b, 0x63, 0x56, 0x76, 0x27, 0xb9, 0xa9, 0x36, 0x4e, 0xad, 0x3e, 0xce, 0xc1, 0x39, 0x89, 0xc7,
0xcf, 0x4a, 0xff, 0xa6, 0x92, 0x23, 0x78, 0x20, 0x8f, 0xc7, 0x17, 0x59, 0x24, 0x27, 0x45, 0x9e,
0x8f, 0x8a, 0x3c, 0x83, 0x81, 0x25, 0xe5, 0x59, 0x1c, 0x93, 0x27, 0xba, 0x0a, 0x35, 0xfc, 0x70,
0x68, 0x7a, 0xb8, 0xe7, 0x9b, 0x36, 0x6e, 0x97, 0x16, 0x95, 0xa5, 0x82, 0x06, 0x1c, 0xb4, 0x6b,
0xda, 0x51, 0x8b, 0x2c, 0x67, 0xb6, 0x48, 0xf5, 0x77, 0x0a, 0x5c, 0x18, 0xd3, 0x92, 0x30, 0x71,
0x0d, 0x5a, 0xec, 0xe6, 0xa1, 0x64, 0xa8, 0xb1, 0x53, 0x81, 0xdf, 0x9c, 0x26, 0xf0, 0x10, 0x5d,
0x1b, 0xa3, 0x8f, 0x1c, 0x32, 0x97, 0xfd, 0x90, 0x87, 0x70, 0x61, 0x03, 0xfb, 0x62, 0x03, 0xfa,
0x0d, 0x93, 0x93, 0x87, 0x80, 0xb8, 0x2f, 0xe5, 0xc6, 0x7c, 0xe9, 0x4f, 0x39, 0xe9, 0x4b, 0x6c,
0xab, 0xae, 0xb3, 0xef, 0xa2, 0xcb, 0x50, 0x95, 0x28, 0xc2, 0x2a, 0x42, 0x00, 0x7a, 0x1d, 0x8a,
0xf4, 0xa4, 0xdc, 0x24, 0x9a, 0xcb, 0xd7, 0xd2, 0xef, 0x14, 0xe1, 0xa9, 0x71, 0x7c, 0xd4, 0x85,
0x26, 0xf1, 0x75, 0xcf, 0xef, 0x0d, 0x5d, 0xc2, 0xf4, 0xcc, 0x0c, 0xa7, 0xb6, 0xac, 0xc6, 0x39,
0xc8, 0x10, 0xb9, 0x45, 0x06, 0xdb, 0x02, 0x53, 0x6b, 0x30, 0xca, 0x60, 0x89, 0xde, 0x87, 0x3a,
0x76, 0x8c, 0x90, 0x51, 0x21, 0x33, 0xa3, 0x1a, 0x76, 0x0c, 0xc9, 0x26, 0xd4, 0x4f, 0x31, 0xbb,
0x7e, 0xbe, 0x56, 0xa0, 0x3d, 0xae, 0xa0, 0xd3, 0x04, 0xca, 0x37, 0x39, 0x11, 0xe6, 0x0a, 0x9a,
0xea, 0xe1, 0x52, 0x49, 0x9a, 0x20, 0x51, 0x4d, 0x78, 0x2e, 0x3c, 0x0d, 0xfb, 0xf2, 0xd4, 0x8c,
0xe5, 0x2b, 0x05, 0xe6, 0x93, 0x7b, 0x9d, 0xe6, 0xde, 0xff, 0x05, 0x45, 0xd3, 0xd9, 0x77, 0x83,
0x6b, 0x2f, 0x4c, 0xf1, 0x33, 0xba, 0x17, 0x47, 0x56, 0x6d, 0xb8, 0xb4, 0x81, 0xfd, 0xae, 0x43,
0xb0, 0xe7, 0xaf, 0x98, 0x8e, 0xe5, 0x0e, 0xb6, 0x75, 0xff, 0xe0, 0x14, 0x3e, 0x12, 0x33, 0xf7,
0x5c, 0xc2, 0xdc, 0xd5, 0xdf, 0x2b, 0x70, 0x39, 0x7d, 0x3f, 0x71, 0xf5, 0x0e, 0x54, 0xf6, 0x4d,
0x6c, 0x19, 0x54, 0x66, 0x0a, 0x93, 0x99, 0x5c, 0x53, 0x5f, 0x19, 0x52, 0x64, 0x71, 0xc3, 0x6b,
0x13, 0x0c, 0x74, 0xc7, 0xf7, 0x4c, 0x67, 0xb0, 0x69, 0x12, 0x5f, 0xe3, 0xf8, 0x11, 0x79, 0xe6,
0xb3, 0x5b, 0xe6, 0x4f, 0x15, 0x58, 0xd8, 0xc0, 0xfe, 0xaa, 0x0c, 0xb5, 0xf4, 0xbb, 0x49, 0x7c,
0xb3, 0x4f, 0x9e, 0x6e, 0x11, 0x91, 0x92, 0x33, 0xd5, 0x9f, 0x29, 0x70, 0x75, 0xe2, 0x61, 0x84,
0xe8, 0x44, 0x28, 0x09, 0x02, 0x6d, 0x7a, 0x28, 0xf9, 0x1f, 0xfc, 0xe8, 0x63, 0xdd, 0x1a, 0xe1,
0x6d, 0xdd, 0xf4, 0x78, 0x28, 0x39, 0x61, 0x60, 0xfd, 0xa3, 0x02, 0x57, 0x36, 0xb0, 0xbf, 0x1d,
0xa4, 0x99, 0x67, 0x28, 0x9d, 0x0c, 0x15, 0xc5, 0x37, 0x5c, 0x99, 0xa9, 0xa7, 0x7d, 0x26, 0xe2,
0x5b, 0x60, 0x7e, 0x10, 0x71, 0xc8, 0x55, 0x5e, 0x0b, 0x08, 0xe1, 0xa9, 0x8f, 0xf3, 0x50, 0xff,
0x58, 0xd4, 0x07, 0x2c, 0x8d, 0x24, 0xe5, 0xa0, 0xa4, 0xcb, 0x21, 0x52, 0x52, 0xa4, 0x55, 0x19,
0x1b, 0xd0, 0x20, 0x18, 0x1f, 0x9e, 0x24, 0x69, 0xd4, 0x29, 0xa1, 0x0c, 0xf6, 0x9b, 0x70, 0x76,
0xe4, 0xec, 0xd3, 0xb2, 0x16, 0x1b, 0xe2, 0x16, 0xbc, 0xba, 0x9c, 0x1d, 0x79, 0xc6, 0x09, 0xd1,
0x07, 0x30, 0x97, 0xe4, 0x55, 0xcc, 0xc4, 0x2b, 0x49, 0x86, 0xba, 0xd0, 0x32, 0x3c, 0x77, 0x38,
0xc4, 0x46, 0x8f, 0x04, 0xac, 0x4a, 0xd9, 0x58, 0x09, 0xba, 0x80, 0x95, 0xfa, 0x13, 0x05, 0xe6,
0x3f, 0xd1, 0xfd, 0xfe, 0xc1, 0x9a, 0x2d, 0x94, 0x73, 0x0a, 0xd3, 0x7e, 0x1b, 0xaa, 0x0f, 0x84,
0x22, 0x82, 0xf8, 0x75, 0x35, 0xe5, 0x40, 0x51, 0x95, 0x6b, 0x21, 0x85, 0xfa, 0xad, 0x02, 0xe7,
0x59, 0x13, 0x11, 0x9c, 0xee, 0xfb, 0x77, 0xb2, 0x19, 0x8d, 0x04, 0xba, 0x09, 0x4d, 0x5b, 0xf7,
0x0e, 0x77, 0x42, 0x9c, 0x22, 0xc3, 0x49, 0x40, 0xd5, 0x87, 0x00, 0x62, 0xb5, 0x45, 0x06, 0x27,
0x38, 0xff, 0x1b, 0x50, 0x16, 0xbb, 0x0a, 0x7f, 0x9b, 0xa5, 0xd8, 0x00, 0x5d, 0xfd, 0x4e, 0x81,
0x66, 0x18, 0x41, 0x99, 0x57, 0x35, 0x21, 0x27, 0x7d, 0x29, 0xd7, 0x5d, 0x43, 0x6f, 0x43, 0x89,
0xb7, 0x8d, 0x82, 0xf7, 0x8d, 0x38, 0x6f, 0xd1, 0x52, 0x46, 0xc2, 0x30, 0x03, 0x68, 0x82, 0x88,
0xca, 0x48, 0x46, 0x1d, 0xde, 0x61, 0xe4, 0xb5, 0x08, 0x04, 0x75, 0x61, 0x2e, 0x5e, 0xb4, 0x05,
0x3e, 0xb3, 0x38, 0x29, 0xda, 0xac, 0xe9, 0xbe, 0xce, 0x82, 0x4d, 0x33, 0x56, 0xb3, 0x11, 0xf5,
0xdf, 0x45, 0xa8, 0x45, 0x6e, 0x39, 0x76, 0x93, 0xa4, 0x4a, 0x73, 0xb3, 0xe3, 0x66, 0x7e, 0xbc,
0x73, 0xb8, 0x01, 0x4d, 0x93, 0xe5, 0xea, 0x9e, 0x30, 0x45, 0x16, 0x5c, 0xab, 0x5a, 0x83, 0x43,
0x85, 0x5f, 0xa0, 0x05, 0xa8, 0x39, 0x23, 0xbb, 0xe7, 0xee, 0xf7, 0x3c, 0xf7, 0x88, 0x88, 0x16,
0xa4, 0xea, 0x8c, 0xec, 0xff, 0xdd, 0xd7, 0xdc, 0x23, 0x12, 0x56, 0xb9, 0xa5, 0x63, 0x56, 0xb9,
0x0b, 0x50, 0xb3, 0xf5, 0x87, 0x94, 0x6b, 0xcf, 0x19, 0xd9, 0xac, 0x3b, 0xc9, 0x6b, 0x55, 0x5b,
0x7f, 0xa8, 0xb9, 0x47, 0xf7, 0x47, 0x36, 0x5a, 0x82, 0x96, 0xa5, 0x13, 0xbf, 0x17, 0x6d, 0x6f,
0x2a, 0xac, 0xbd, 0x69, 0x52, 0xf8, 0xfb, 0x61, 0x8b, 0x33, 0x5e, 0x2f, 0x57, 0x4f, 0x51, 0x2f,
0x1b, 0xb6, 0x15, 0x32, 0x82, 0xec, 0xf5, 0xb2, 0x61, 0x5b, 0x92, 0xcd, 0x1b, 0x50, 0xde, 0x63,
0x15, 0x10, 0x69, 0xd7, 0x26, 0x46, 0xa8, 0x75, 0x5a, 0xfc, 0xf0, 0x42, 0x49, 0x0b, 0xd0, 0xd1,
0x5b, 0x50, 0x65, 0xa9, 0x87, 0xd1, 0xd6, 0x33, 0xd1, 0x86, 0x04, 0x94, 0xda, 0xc0, 0x96, 0xaf,
0x33, 0xea, 0x46, 0x36, 0x6a, 0x49, 0x80, 0x5e, 0x81, 0x73, 0x7d, 0x0f, 0xeb, 0x3e, 0x36, 0x56,
0x1e, 0xad, 0xba, 0xf6, 0x50, 0x67, 0xc6, 0xd4, 0x6e, 0x2e, 0x2a, 0x4b, 0x15, 0x2d, 0xed, 0x13,
0x0d, 0x0c, 0x7d, 0xb9, 0x5a, 0xf7, 0x5c, 0xbb, 0x3d, 0xc7, 0x03, 0x43, 0x1c, 0x8a, 0xae, 0x00,
0x04, 0xa1, 0x5b, 0xf7, 0xdb, 0x2d, 0xa6, 0xc5, 0xaa, 0x80, 0xbc, 0xe7, 0xab, 0x5f, 0xc2, 0xf9,
0xd0, 0x42, 0x22, 0xda, 0x18, 0x57, 0xac, 0x72, 0x52, 0xc5, 0x4e, 0xaf, 0x5d, 0xff, 0x5c, 0x80,
0xf9, 0x1d, 0xfd, 0x01, 0x7e, 0xfa, 0x65, 0x72, 0xa6, 0x78, 0xbc, 0x09, 0x67, 0x59, 0x65, 0xbc,
0x1c, 0x39, 0xcf, 0x94, 0x0c, 0x1c, 0x55, 0xe7, 0x38, 0x21, 0x7a, 0x97, 0x96, 0x0e, 0xb8, 0x7f,
0xb8, 0xed, 0x9a, 0x61, 0xf6, 0xbd, 0x92, 0xc2, 0x67, 0x55, 0x62, 0x69, 0x51, 0x0a, 0xb4, 0x3d,
0x1e, 0xda, 0x78, 0xde, 0x7d, 0x61, 0x6a, 0xff, 0x15, 0x4a, 0x3f, 0x19, 0xe1, 0x50, 0x1b, 0xca,
0x22, 0xbb, 0x33, 0xbf, 0xaf, 0x68, 0xc1, 0x12, 0x6d, 0xc3, 0x39, 0x7e, 0x83, 0x1d, 0x61, 0xd4,
0xfc, 0xf2, 0x95, 0x4c, 0x97, 0x4f, 0x23, 0x8d, 0xfb, 0x44, 0xf5, 0xb8, 0x3e, 0xd1, 0x86, 0xb2,
0xb0, 0x53, 0x16, 0x0b, 0x2a, 0x5a, 0xb0, 0xa4, 0x4d, 0x04, 0x84, 0x12, 0x9b, 0x31, 0x0b, 0x78,
0x07, 0x2a, 0xd2, 0x86, 0x73, 0x99, 0x6d, 0x58, 0xd2, 0x24, 0xa3, 0x70, 0x3e, 0x11, 0x85, 0xd5,
0x7f, 0x28, 0x50, 0x5f, 0xa3, 0x87, 0xde, 0x74, 0x07, 0x2c, 0x67, 0xdc, 0x80, 0xa6, 0x87, 0xfb,
0xae, 0x67, 0xf4, 0xb0, 0xe3, 0x7b, 0x26, 0xe6, 0xfd, 0x66, 0x41, 0x6b, 0x70, 0xe8, 0xfb, 0x1c,
0x48, 0xd1, 0x68, 0x60, 0x25, 0xbe, 0x6e, 0x0f, 0x7b, 0xfb, 0xd4, 0x81, 0x73, 0x1c, 0x4d, 0x42,
0x99, 0xff, 0x5e, 0x83, 0x7a, 0x88, 0xe6, 0xbb, 0x6c, 0xff, 0x82, 0x56, 0x93, 0xb0, 0x5d, 0x17,
0x3d, 0x0f, 0x4d, 0x26, 0xb5, 0x9e, 0xe5, 0x0e, 0x7a, 0xb4, 0x37, 0x13, 0xe9, 0xa4, 0x6e, 0x88,
0x63, 0x51, 0x6d, 0xc4, 0xb1, 0x88, 0xf9, 0x05, 0x16, 0x09, 0x45, 0x62, 0xed, 0x98, 0x5f, 0x60,
0x9a, 0xcd, 0x1b, 0x34, 0x3b, 0xde, 0x77, 0x0d, 0xbc, 0x7b, 0xc2, 0x5a, 0x22, 0xc3, 0x5c, 0xee,
0x32, 0x54, 0xe5, 0x0d, 0xc4, 0x95, 0x42, 0x00, 0x5a, 0x87, 0x66, 0x50, 0x66, 0xf6, 0x78, 0xf7,
0x50, 0x98, 0x58, 0xdb, 0x45, 0xf2, 0x1b, 0xd1, 0x1a, 0x01, 0x19, 0x5b, 0xaa, 0xeb, 0x50, 0x8f,
0x7e, 0xa6, 0xbb, 0xee, 0x24, 0x0d, 0x45, 0x02, 0xa8, 0xbd, 0xdd, 0x1f, 0xd9, 0x54, 0xa7, 0x22,
0x74, 0x04, 0x4b, 0xf5, 0x2b, 0x05, 0x1a, 0x22, 0x29, 0xef, 0xc8, 0xb9, 0x31, 0xbb, 0x9a, 0xc2,
0xae, 0xc6, 0xfe, 0x46, 0xff, 0x1d, 0x1f, 0x3a, 0x3d, 0x9f, 0xea, 0xe6, 0x8c, 0x09, 0xab, 0x7f,
0x63, 0x19, 0x39, 0x4b, 0xb7, 0xfa, 0x98, 0x1a, 0x9a, 0x50, 0x0d, 0x33, 0xb4, 0x36, 0x94, 0x75,
0xc3, 0xf0, 0x30, 0x21, 0xe2, 0x1c, 0xc1, 0x92, 0x7e, 0x79, 0x80, 0x3d, 0x12, 0x98, 0x7c, 0x5e,
0x0b, 0x96, 0xe8, 0x2d, 0xa8, 0xc8, 0x82, 0x39, 0x9f, 0x56, 0x24, 0x45, 0xcf, 0x29, 0xba, 0x2b,
0x49, 0xa1, 0x7e, 0x93, 0x83, 0xa6, 0x10, 0xd8, 0x8a, 0xc8, 0x9a, 0xd3, 0x9d, 0x6f, 0x05, 0xea,
0xfb, 0xa1, 0x77, 0x4f, 0x9b, 0xa2, 0x44, 0x83, 0x40, 0x8c, 0x66, 0x96, 0x03, 0xc6, 0xf3, 0x76,
0xe1, 0x54, 0x79, 0xbb, 0x78, 0xcc, 0x18, 0xa5, 0xfe, 0x00, 0x6a, 0x91, 0x2f, 0x2c, 0xb8, 0xf2,
0xb9, 0x8a, 0x10, 0x45, 0xb0, 0x44, 0xf7, 0xc2, 0xb2, 0x84, 0xcb, 0xe0, 0x62, 0xca, 0x26, 0x89,
0x8a, 0x44, 0xfd, 0x83, 0x02, 0x25, 0xc1, 0xf9, 0x2a, 0xd4, 0x44, 0x34, 0x61, 0x25, 0x1b, 0xe7,
0x0e, 0x02, 0x44, 0x6b, 0xb6, 0x27, 0x17, 0x4e, 0x2e, 0x42, 0x25, 0x11, 0x48, 0xca, 0x22, 0xa2,
0x07, 0x9f, 0x22, 0xd1, 0x83, 0x7e, 0x62, 0x81, 0xe3, 0x5b, 0x85, 0xcd, 0x84, 0x35, 0xdc, 0x77,
0x1f, 0x60, 0xef, 0xd1, 0xe9, 0x27, 0x6f, 0x6f, 0x46, 0x2c, 0x35, 0x63, 0x6b, 0x27, 0x09, 0xd0,
0x9b, 0xa1, 0xb8, 0xf3, 0x69, 0x83, 0x87, 0x68, 0xe8, 0x10, 0x76, 0x16, 0x8a, 0xfd, 0xe7, 0x7c,
0x86, 0x18, 0xbf, 0xca, 0x49, 0x4b, 0x92, 0x27, 0xd2, 0x31, 0xa8, 0xbf, 0x54, 0xe0, 0xe2, 0x06,
0xf6, 0xd7, 0xe3, 0x7d, 0xf9, 0xb3, 0x3e, 0x95, 0x0d, 0x9d, 0xb4, 0x43, 0x9d, 0x46, 0xeb, 0x1d,
0xa8, 0xc8, 0x09, 0x03, 0x9f, 0xee, 0xca, 0xb5, 0xfa, 0x63, 0x05, 0xda, 0x62, 0x17, 0xb6, 0x27,
0xad, 0x86, 0x2d, 0xec, 0x63, 0xe3, 0xfb, 0x6e, 0x79, 0xff, 0xae, 0x40, 0x2b, 0x1a, 0xca, 0x59,
0x34, 0x7e, 0x0d, 0x8a, 0x6c, 0xb2, 0x20, 0x4e, 0x30, 0xd3, 0x58, 0x39, 0x36, 0x0d, 0x19, 0xac,
0x42, 0xdb, 0x95, 0x59, 0x47, 0x2c, 0xc3, 0x7c, 0x92, 0x3f, 0x7e, 0x3e, 0x11, 0xf9, 0xd5, 0x1d,
0x51, 0xbe, 0x7c, 0x72, 0x17, 0x02, 0xd4, 0xaf, 0x73, 0xd0, 0x0e, 0x5b, 0x89, 0xef, 0x3d, 0xa0,
0x4f, 0x28, 0x34, 0xf3, 0x4f, 0xa8, 0xd0, 0x2c, 0x1c, 0x37, 0x88, 0xff, 0x2d, 0x07, 0xcd, 0x50,
0x1c, 0xdb, 0x96, 0xee, 0xa0, 0x79, 0x28, 0x0d, 0x2d, 0x3d, 0x9c, 0x08, 0x8a, 0x15, 0xda, 0x91,
0x95, 0x49, 0x5c, 0x00, 0x2f, 0xa5, 0x29, 0x67, 0x82, 0x84, 0xb5, 0x04, 0x0b, 0xda, 0xa2, 0xf1,
0x22, 0x9f, 0x35, 0xda, 0xa2, 0x1a, 0xe2, 0x56, 0x40, 0x7b, 0xec, 0xdb, 0x80, 0x84, 0xea, 0x7a,
0xa6, 0xd3, 0x23, 0xb8, 0xef, 0x3a, 0x06, 0x57, 0x6a, 0x51, 0x6b, 0x89, 0x2f, 0x5d, 0x67, 0x87,
0xc3, 0xd1, 0x6b, 0x50, 0xf0, 0x1f, 0x0d, 0x79, 0x78, 0x6e, 0xa6, 0x86, 0xbd, 0xf0, 0x5c, 0xbb,
0x8f, 0x86, 0x58, 0x63, 0xe8, 0x68, 0x01, 0x80, 0xb2, 0xf2, 0x3d, 0xfd, 0x01, 0xb6, 0x82, 0xdf,
0x32, 0x43, 0x08, 0x35, 0xd3, 0x60, 0x56, 0x51, 0xe6, 0x39, 0x41, 0x2c, 0xd5, 0xbf, 0xe4, 0xa0,
0x15, 0xb2, 0xd4, 0x30, 0x19, 0x59, 0xfe, 0x44, 0xf9, 0x4d, 0x6f, 0xd0, 0x66, 0x65, 0xfa, 0x77,
0xa1, 0x26, 0xe6, 0x26, 0xc7, 0x50, 0x34, 0x70, 0x92, 0xcd, 0x29, 0x96, 0x57, 0x7c, 0x42, 0x96,
0x57, 0x3a, 0xae, 0xe5, 0xed, 0xc0, 0x7c, 0x10, 0xd0, 0x42, 0x84, 0x2d, 0xec, 0xeb, 0x53, 0x2a,
0x89, 0xab, 0x50, 0xe3, 0x89, 0x8a, 0x67, 0x68, 0x5e, 0x5c, 0xc3, 0x9e, 0xec, 0x3a, 0xd5, 0x1f,
0xc2, 0x79, 0x16, 0x10, 0x92, 0xe3, 0xd5, 0x2c, 0xb3, 0x6e, 0x55, 0x96, 0xee, 0xb4, 0x4c, 0xe7,
0xd6, 0x5d, 0xd5, 0x62, 0x30, 0x75, 0x13, 0x9e, 0x4b, 0xf0, 0x3f, 0x45, 0xc0, 0x57, 0xff, 0xaa,
0xc0, 0xc5, 0x35, 0xcf, 0x1d, 0x7e, 0x6c, 0x7a, 0xfe, 0x48, 0xb7, 0xe2, 0x03, 0xfb, 0xa7, 0xd3,
0x7c, 0x7c, 0x10, 0xc9, 0x31, 0x3c, 0xe8, 0xdc, 0x4e, 0x51, 0xd9, 0xf8, 0xa1, 0x84, 0xaa, 0x22,
0x19, 0xe9, 0x5f, 0xf9, 0xb4, 0xc3, 0x0b, 0xbc, 0x19, 0x91, 0x34, 0x4b, 0x0a, 0x4e, 0x9d, 0x46,
0xe4, 0x4f, 0x3a, 0x8d, 0x98, 0x60, 0xfd, 0x85, 0x27, 0x64, 0xfd, 0xc7, 0x2d, 0x9e, 0xd1, 0x07,
0x10, 0x9f, 0x14, 0xb1, 0xb0, 0x73, 0xa2, 0x11, 0xd3, 0x0a, 0x40, 0x38, 0x35, 0x11, 0xaf, 0x2d,
0xb2, 0xb0, 0x89, 0x50, 0x51, 0x6d, 0xc9, 0x48, 0xc3, 0xa6, 0x9d, 0xb1, 0x2e, 0xff, 0x43, 0xe8,
0xa4, 0x59, 0xe9, 0x69, 0x2c, 0xff, 0x9b, 0x1c, 0x40, 0xd7, 0x1e, 0xba, 0x9e, 0xbf, 0xab, 0x93,
0xc3, 0x93, 0x95, 0x4b, 0xd7, 0xa1, 0x11, 0x1a, 0x4c, 0xcf, 0x34, 0x52, 0xac, 0xc8, 0xa0, 0x2e,
0x21, 0xab, 0x36, 0x8a, 0x33, 0x56, 0xc9, 0x19, 0xe8, 0x12, 0x54, 0x3d, 0xf7, 0xa8, 0x47, 0x3d,
0xc8, 0x60, 0xa9, 0xa5, 0xa2, 0x55, 0x3c, 0xf7, 0x88, 0xfa, 0x95, 0x81, 0x2e, 0x40, 0xd9, 0xd7,
0xc9, 0x21, 0x25, 0xe5, 0x45, 0x7f, 0x89, 0x2e, 0xbb, 0x06, 0x3a, 0x0f, 0xc5, 0x7d, 0xd3, 0xc2,
0x3c, 0xf0, 0x55, 0x35, 0xbe, 0x40, 0xaf, 0x07, 0x3f, 0x99, 0x97, 0x33, 0xff, 0xe4, 0xc7, 0x7f,
0x35, 0xff, 0x56, 0x81, 0xb9, 0x50, 0x20, 0xac, 0x9e, 0x41, 0xef, 0xf0, 0xe6, 0x0e, 0xaf, 0xba,
0x06, 0x8f, 0x02, 0xcd, 0x09, 0x53, 0x7d, 0x4e, 0xc8, 0x8b, 0xa0, 0x90, 0x64, 0x5a, 0x3d, 0x49,
0xef, 0x45, 0x2f, 0x6d, 0x1a, 0xc1, 0x8f, 0x0a, 0x25, 0xcf, 0x3d, 0xea, 0x1a, 0x24, 0x90, 0x06,
0x7f, 0x39, 0xc4, 0xab, 0x27, 0x2a, 0x8d, 0x55, 0xf6, 0x78, 0xe8, 0x3a, 0x34, 0xb0, 0xe7, 0xb9,
0x5e, 0xcf, 0xc6, 0x84, 0xe8, 0x03, 0x9e, 0x69, 0xab, 0x5a, 0x9d, 0x01, 0xb7, 0x38, 0x4c, 0xfd,
0x2e, 0x07, 0xcd, 0xf0, 0x2a, 0xc1, 0x4f, 0x09, 0xa6, 0x11, 0xfc, 0x94, 0x60, 0x1a, 0x34, 0xeb,
0x7b, 0x3c, 0xca, 0x85, 0x7a, 0xab, 0x0a, 0x48, 0xd7, 0xa0, 0x61, 0x9e, 0xfa, 0x8e, 0xe3, 0x1a,
0x38, 0xd4, 0x19, 0x04, 0xa0, 0xae, 0x31, 0xae, 0xfa, 0x42, 0x06, 0xd5, 0x17, 0xc7, 0x55, 0x3f,
0x0f, 0xa5, 0xbd, 0x51, 0xff, 0x10, 0xfb, 0xcc, 0xfd, 0xaa, 0x9a, 0x58, 0xc5, 0x4d, 0xa2, 0x9c,
0x30, 0x09, 0xa9, 0xf9, 0x4a, 0x54, 0xf3, 0x97, 0xa0, 0xca, 0x47, 0xd5, 0x3d, 0x9f, 0xb0, 0x1f,
0x02, 0xf2, 0x5a, 0x85, 0x03, 0x76, 0x09, 0x7a, 0x23, 0x28, 0x67, 0x53, 0x07, 0xfb, 0x2c, 0x4e,
0x24, 0x94, 0x2f, 0x8a, 0x59, 0xf5, 0x33, 0x40, 0xe1, 0x97, 0xd3, 0xb5, 0x17, 0x09, 0xa9, 0xe6,
0x92, 0x52, 0xbd, 0xf5, 0x2b, 0x05, 0xce, 0x8e, 0x55, 0xd5, 0xa8, 0x09, 0xf0, 0x91, 0xd3, 0x17,
0xed, 0x46, 0xeb, 0x0c, 0xaa, 0x43, 0x25, 0x68, 0x3e, 0x5a, 0x0a, 0xaa, 0x41, 0x79, 0xd7, 0x65,
0xd8, 0xad, 0x1c, 0x6a, 0x41, 0x9d, 0x13, 0x8e, 0xfa, 0x7d, 0x4c, 0x48, 0x2b, 0x2f, 0x21, 0xeb,
0xba, 0x69, 0x8d, 0x3c, 0xdc, 0x2a, 0xa0, 0x06, 0x54, 0x77, 0x5d, 0x0d, 0x5b, 0x58, 0x27, 0xb8,
0x55, 0x44, 0x08, 0x9a, 0x62, 0x11, 0x10, 0x95, 0x22, 0xb0, 0x80, 0xac, 0x7c, 0x6b, 0x3f, 0x5a,
0xa6, 0xd2, 0xda, 0x0d, 0x5d, 0x80, 0x73, 0x1f, 0x39, 0x06, 0xde, 0x37, 0x1d, 0x6c, 0x84, 0x9f,
0x5a, 0x67, 0xd0, 0x39, 0x98, 0xeb, 0x3a, 0x0e, 0xf6, 0x22, 0x40, 0x85, 0x02, 0xb7, 0xb0, 0x37,
0xc0, 0x11, 0x60, 0x0e, 0x9d, 0x85, 0xc6, 0x96, 0xf9, 0x30, 0x02, 0xca, 0x2f, 0xff, 0xf3, 0x1c,
0x54, 0xd7, 0x74, 0x5f, 0x5f, 0x75, 0x5d, 0xcf, 0x40, 0x43, 0x40, 0xec, 0x8d, 0x84, 0x3d, 0x74,
0x1d, 0xf9, 0x98, 0x08, 0xbd, 0x32, 0x21, 0xba, 0x8e, 0xa3, 0x8a, 0x54, 0xde, 0xb9, 0x39, 0x81,
0x22, 0x81, 0xae, 0x9e, 0x41, 0x36, 0xdb, 0x91, 0xd6, 0xbe, 0xbb, 0x66, 0xff, 0x30, 0xf8, 0x35,
0x6c, 0xca, 0x8e, 0x09, 0xd4, 0x60, 0xc7, 0xc4, 0x1b, 0x25, 0xb1, 0xe0, 0x0f, 0x59, 0x02, 0x33,
0x52, 0xcf, 0xa0, 0xcf, 0xe1, 0xfc, 0x06, 0xf6, 0xc3, 0xb7, 0x0b, 0xc1, 0x86, 0xcb, 0x93, 0x37,
0x1c, 0x43, 0x3e, 0xe6, 0x96, 0x9b, 0x50, 0x64, 0x1d, 0x2c, 0x4a, 0xeb, 0x12, 0xa3, 0x2f, 0x6a,
0x3b, 0x8b, 0x93, 0x11, 0x24, 0xb7, 0xcf, 0x60, 0x2e, 0xf1, 0x62, 0x10, 0xbd, 0x98, 0x42, 0x96,
0xfe, 0xf6, 0xb3, 0x73, 0x2b, 0x0b, 0xaa, 0xdc, 0x6b, 0x00, 0xcd, 0xf8, 0x0b, 0x0b, 0xb4, 0x94,
0x42, 0x9f, 0xfa, 0xda, 0xab, 0xf3, 0x62, 0x06, 0x4c, 0xb9, 0x91, 0x0d, 0xad, 0xe4, 0x0b, 0x36,
0x74, 0x6b, 0x2a, 0x83, 0xb8, 0xb9, 0xbd, 0x94, 0x09, 0x57, 0x6e, 0xf7, 0x88, 0x19, 0xc1, 0xd8,
0x0b, 0x2a, 0x74, 0x27, 0x9d, 0xcd, 0xa4, 0xa7, 0x5d, 0x9d, 0xbb, 0x99, 0xf1, 0xe5, 0xd6, 0x3f,
0xe2, 0x93, 0xb3, 0xb4, 0x57, 0x48, 0xe8, 0xd5, 0x74, 0x76, 0x53, 0x9e, 0x4f, 0x75, 0x96, 0x8f,
0x43, 0x22, 0x0f, 0xf1, 0x25, 0x1b, 0x79, 0xa5, 0xbc, 0xe4, 0x49, 0xfa, 0x5d, 0xc0, 0x6f, 0xf2,
0x13, 0xa5, 0xce, 0xab, 0xc7, 0xa0, 0x90, 0x07, 0x70, 0x93, 0x6f, 0x04, 0x03, 0x37, 0xbc, 0x3b,
0xd3, 0x6a, 0x4e, 0xe6, 0x83, 0x9f, 0xc2, 0x5c, 0xe2, 0x77, 0xc7, 0x54, 0xaf, 0x49, 0xff, 0x6d,
0xb2, 0x33, 0x2d, 0xdb, 0x70, 0x97, 0x4c, 0x4c, 0x10, 0xd1, 0x04, 0xeb, 0x4f, 0x99, 0x32, 0x76,
0x6e, 0x65, 0x41, 0x95, 0x17, 0x21, 0x2c, 0x5c, 0x26, 0xa6, 0x70, 0xe8, 0x76, 0x3a, 0x8f, 0xf4,
0x09, 0x62, 0xe7, 0xe5, 0x8c, 0xd8, 0x72, 0xd3, 0x1e, 0xc0, 0x06, 0xf6, 0xb7, 0xb0, 0xef, 0x51,
0x1b, 0xb9, 0x99, 0x2a, 0xf2, 0x10, 0x21, 0xd8, 0xe6, 0x85, 0x99, 0x78, 0x72, 0x83, 0xff, 0x03,
0x14, 0xa4, 0xd8, 0xc8, 0xaf, 0xde, 0xd7, 0xa7, 0xce, 0x33, 0xf8, 0xf0, 0x61, 0x96, 0x6e, 0x3e,
0x87, 0xd6, 0x96, 0xee, 0xd0, 0x2a, 0x3e, 0xe4, 0x7b, 0x3b, 0xf5, 0x60, 0x49, 0xb4, 0x09, 0xd2,
0x9a, 0x88, 0x2d, 0x2f, 0x73, 0x24, 0x73, 0xa8, 0x2e, 0x5d, 0x10, 0x27, 0x63, 0x4b, 0x28, 0x8d,
0x04, 0xe2, 0x84, 0xd8, 0x32, 0x05, 0x5f, 0x6e, 0xfc, 0x58, 0x61, 0x2f, 0x51, 0x13, 0x08, 0x9f,
0x98, 0xfe, 0xc1, 0xb6, 0xa5, 0x3b, 0x24, 0xcb, 0x11, 0x18, 0xe2, 0x31, 0x8e, 0x20, 0xf0, 0xe5,
0x11, 0x0c, 0x68, 0xc4, 0xc6, 0x05, 0x28, 0xed, 0xa7, 0xeb, 0xb4, 0x81, 0x45, 0x67, 0x69, 0x36,
0xa2, 0xdc, 0xe5, 0x00, 0x1a, 0x81, 0xbd, 0x72, 0xe1, 0xbe, 0x38, 0xe9, 0xa4, 0x21, 0xce, 0x04,
0x77, 0x4b, 0x47, 0x8d, 0xba, 0xdb, 0x78, 0x27, 0x88, 0xb2, 0x4d, 0x10, 0xa6, 0xb9, 0xdb, 0xe4,
0xf6, 0x52, 0x3d, 0x83, 0xb6, 0xa1, 0xc4, 0x4b, 0x60, 0x74, 0x65, 0x6a, 0xdd, 0xdc, 0xb9, 0x31,
0xf5, 0x73, 0xc8, 0x71, 0xf9, 0x37, 0x45, 0xa8, 0x04, 0xbf, 0x26, 0x3e, 0x83, 0x1a, 0xef, 0x19,
0x14, 0x5d, 0x9f, 0xc2, 0x5c, 0xe2, 0xe1, 0x61, 0x6a, 0x4c, 0x4e, 0x7f, 0x9c, 0x38, 0x2b, 0xa8,
0x7c, 0x22, 0xfe, 0x1d, 0x49, 0xc6, 0xdf, 0x17, 0x26, 0x15, 0x6e, 0xc9, 0xd0, 0x3b, 0x83, 0xf1,
0x53, 0x0f, 0xb4, 0xf7, 0x01, 0x22, 0x81, 0x70, 0xfa, 0xc0, 0x98, 0xfa, 0xf6, 0xac, 0x03, 0xaf,
0x67, 0x35, 0xd5, 0xe9, 0x7c, 0x56, 0xee, 0xfd, 0xff, 0xab, 0x03, 0xd3, 0x3f, 0x18, 0xed, 0xd1,
0x2f, 0x77, 0x39, 0xea, 0xcb, 0xa6, 0x2b, 0xfe, 0xba, 0x1b, 0x58, 0xc6, 0x5d, 0x46, 0x7d, 0x97,
0x32, 0x1f, 0xee, 0xed, 0x95, 0xd8, 0xea, 0xde, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff, 0xc0, 0xfc,
0x98, 0x50, 0xf8, 0x36, 0x00, 0x00,
0xd5, 0x5e, 0xde, 0x44, 0x1e, 0x5e, 0x44, 0x8d, 0x1d, 0x99, 0xa6, 0x6d, 0x59, 0x5e, 0xc7, 0x8e,
0xe2, 0x38, 0x76, 0x22, 0x7f, 0x41, 0x82, 0x2f, 0x37, 0x44, 0x52, 0xa4, 0x10, 0x9f, 0xe4, 0x4f,
0x59, 0x29, 0x71, 0xd1, 0x14, 0x25, 0x56, 0xdc, 0x11, 0xb5, 0xd1, 0x5e, 0x98, 0x9d, 0xa5, 0x65,
0xe7, 0x25, 0x46, 0x03, 0x14, 0x68, 0x11, 0xa4, 0x2d, 0xfa, 0x54, 0xa0, 0x0f, 0x45, 0x81, 0x02,
0xbd, 0xbc, 0xb4, 0x8f, 0x6d, 0xd1, 0xf7, 0x20, 0xfd, 0x21, 0x45, 0x5f, 0xfa, 0x1b, 0x8a, 0xb9,
0xec, 0xec, 0x85, 0x4b, 0x72, 0x25, 0xd9, 0xf1, 0x9b, 0xe6, 0xec, 0x39, 0x33, 0x67, 0xce, 0xfd,
0x1c, 0x8e, 0xa0, 0x69, 0xe8, 0xbe, 0xde, 0xed, 0xb9, 0xae, 0x67, 0xdc, 0x1e, 0x78, 0xae, 0xef,
0xa2, 0x39, 0xdb, 0xb4, 0x1e, 0x0c, 0x09, 0x5f, 0xdd, 0xa6, 0x9f, 0xdb, 0xb5, 0x9e, 0x6b, 0xdb,
0xae, 0xc3, 0x41, 0xed, 0x86, 0xe9, 0xf8, 0xd8, 0x73, 0x74, 0x4b, 0xac, 0x6b, 0x51, 0x82, 0x76,
0x8d, 0xf4, 0x0e, 0xb0, 0xad, 0xf3, 0x95, 0xfa, 0x10, 0x6a, 0xeb, 0xd6, 0x90, 0x1c, 0x68, 0xf8,
0xb3, 0x21, 0x26, 0x3e, 0x7a, 0x05, 0x0a, 0x7b, 0x3a, 0xc1, 0x2d, 0x65, 0x51, 0x59, 0xaa, 0x2e,
0x5f, 0xba, 0x1d, 0x3b, 0x4b, 0x9c, 0xb2, 0x45, 0xfa, 0x2b, 0x3a, 0xc1, 0x1a, 0xc3, 0x44, 0x08,
0x0a, 0xc6, 0x5e, 0x67, 0xad, 0x95, 0x5b, 0x54, 0x96, 0xf2, 0x1a, 0xfb, 0x1b, 0xa9, 0x50, 0xeb,
0xb9, 0x96, 0x85, 0x7b, 0xbe, 0xe9, 0x3a, 0x9d, 0xb5, 0x56, 0x81, 0x7d, 0x8b, 0xc1, 0xd4, 0x5f,
0x2b, 0x50, 0x17, 0x47, 0x93, 0x81, 0xeb, 0x10, 0x8c, 0xee, 0x42, 0x89, 0xf8, 0xba, 0x3f, 0x24,
0xe2, 0xf4, 0x8b, 0xa9, 0xa7, 0xef, 0x30, 0x14, 0x4d, 0xa0, 0x66, 0x3a, 0x3e, 0x3f, 0x7a, 0x3c,
0x5a, 0x00, 0x20, 0xb8, 0x6f, 0x63, 0xc7, 0xef, 0xac, 0x91, 0x56, 0x61, 0x31, 0xbf, 0x94, 0xd7,
0x22, 0x10, 0xf5, 0x17, 0x0a, 0x34, 0x77, 0x82, 0x65, 0x20, 0x9d, 0x73, 0x50, 0xec, 0xb9, 0x43,
0xc7, 0x67, 0x0c, 0xd6, 0x35, 0xbe, 0x40, 0x57, 0xa1, 0xd6, 0x3b, 0xd0, 0x1d, 0x07, 0x5b, 0x5d,
0x47, 0xb7, 0x31, 0x63, 0xa5, 0xa2, 0x55, 0x05, 0xec, 0x9e, 0x6e, 0xe3, 0x4c, 0x1c, 0x2d, 0x42,
0x75, 0xa0, 0x7b, 0xbe, 0x19, 0x93, 0x59, 0x14, 0xa4, 0xfe, 0x46, 0x81, 0xf9, 0xf7, 0x08, 0x31,
0xfb, 0xce, 0x08, 0x67, 0xf3, 0x50, 0x72, 0x5c, 0x03, 0x77, 0xd6, 0x18, 0x6b, 0x79, 0x4d, 0xac,
0xd0, 0x45, 0xa8, 0x0c, 0x30, 0xf6, 0xba, 0x9e, 0x6b, 0x05, 0x8c, 0x95, 0x29, 0x40, 0x73, 0x2d,
0x8c, 0x3e, 0x84, 0x39, 0x92, 0xd8, 0x88, 0xb4, 0xf2, 0x8b, 0xf9, 0xa5, 0xea, 0xf2, 0xb5, 0xdb,
0x23, 0x56, 0x76, 0x3b, 0x79, 0xa8, 0x36, 0x4a, 0xad, 0x3e, 0xce, 0xc1, 0x59, 0x89, 0xc7, 0x79,
0xa5, 0x7f, 0x53, 0xc9, 0x11, 0xdc, 0x97, 0xec, 0xf1, 0x45, 0x16, 0xc9, 0x49, 0x91, 0xe7, 0xa3,
0x22, 0xcf, 0x60, 0x60, 0x49, 0x79, 0x16, 0x47, 0xe4, 0x89, 0xae, 0x40, 0x15, 0x3f, 0x1c, 0x98,
0x1e, 0xee, 0xfa, 0xa6, 0x8d, 0x5b, 0xa5, 0x45, 0x65, 0xa9, 0xa0, 0x01, 0x07, 0xed, 0x9a, 0x76,
0xd4, 0x22, 0x67, 0x32, 0x5b, 0xa4, 0xfa, 0x5b, 0x05, 0xce, 0x8f, 0x68, 0x49, 0x98, 0xb8, 0x06,
0x4d, 0x76, 0xf3, 0x50, 0x32, 0xd4, 0xd8, 0xa9, 0xc0, 0x6f, 0x4c, 0x12, 0x78, 0x88, 0xae, 0x8d,
0xd0, 0x47, 0x98, 0xcc, 0x65, 0x67, 0xf2, 0x10, 0xce, 0x6f, 0x60, 0x5f, 0x1c, 0x40, 0xbf, 0x61,
0x72, 0xf2, 0x10, 0x10, 0xf7, 0xa5, 0xdc, 0x88, 0x2f, 0xfd, 0x39, 0x27, 0x7d, 0x89, 0x1d, 0xd5,
0x71, 0xf6, 0x5d, 0x74, 0x09, 0x2a, 0x12, 0x45, 0x58, 0x45, 0x08, 0x40, 0xaf, 0x43, 0x91, 0x72,
0xca, 0x4d, 0xa2, 0xb1, 0x7c, 0x35, 0xfd, 0x4e, 0x91, 0x3d, 0x35, 0x8e, 0x8f, 0x3a, 0xd0, 0x20,
0xbe, 0xee, 0xf9, 0xdd, 0x81, 0x4b, 0x98, 0x9e, 0x99, 0xe1, 0x54, 0x97, 0xd5, 0xf8, 0x0e, 0x32,
0x44, 0x6e, 0x91, 0xfe, 0xb6, 0xc0, 0xd4, 0xea, 0x8c, 0x32, 0x58, 0xa2, 0xf7, 0xa1, 0x86, 0x1d,
0x23, 0xdc, 0xa8, 0x90, 0x79, 0xa3, 0x2a, 0x76, 0x0c, 0xb9, 0x4d, 0xa8, 0x9f, 0x62, 0x76, 0xfd,
0x7c, 0xa5, 0x40, 0x6b, 0x54, 0x41, 0xa7, 0x09, 0x94, 0x6f, 0x72, 0x22, 0xcc, 0x15, 0x34, 0xd1,
0xc3, 0xa5, 0x92, 0x34, 0x41, 0xa2, 0x9a, 0xf0, 0x5c, 0xc8, 0x0d, 0xfb, 0xf2, 0xd4, 0x8c, 0xe5,
0x4b, 0x05, 0xe6, 0x93, 0x67, 0x9d, 0xe6, 0xde, 0xff, 0x03, 0x45, 0xd3, 0xd9, 0x77, 0x83, 0x6b,
0x2f, 0x4c, 0xf0, 0x33, 0x7a, 0x16, 0x47, 0x56, 0x6d, 0xb8, 0xb8, 0x81, 0xfd, 0x8e, 0x43, 0xb0,
0xe7, 0xaf, 0x98, 0x8e, 0xe5, 0xf6, 0xb7, 0x75, 0xff, 0xe0, 0x14, 0x3e, 0x12, 0x33, 0xf7, 0x5c,
0xc2, 0xdc, 0xd5, 0xdf, 0x2b, 0x70, 0x29, 0xfd, 0x3c, 0x71, 0xf5, 0x36, 0x94, 0xf7, 0x4d, 0x6c,
0x19, 0x54, 0x66, 0x0a, 0x93, 0x99, 0x5c, 0x53, 0x5f, 0x19, 0x50, 0x64, 0x71, 0xc3, 0xab, 0x63,
0x0c, 0x74, 0xc7, 0xf7, 0x4c, 0xa7, 0xbf, 0x69, 0x12, 0x5f, 0xe3, 0xf8, 0x11, 0x79, 0xe6, 0xb3,
0x5b, 0xe6, 0x4f, 0x15, 0x58, 0xd8, 0xc0, 0xfe, 0xaa, 0x0c, 0xb5, 0xf4, 0xbb, 0x49, 0x7c, 0xb3,
0x47, 0x9e, 0x6e, 0x11, 0x91, 0x92, 0x33, 0xd5, 0x9f, 0x29, 0x70, 0x65, 0x2c, 0x33, 0x42, 0x74,
0x22, 0x94, 0x04, 0x81, 0x36, 0x3d, 0x94, 0xfc, 0x1f, 0x7e, 0xf4, 0xb1, 0x6e, 0x0d, 0xf1, 0xb6,
0x6e, 0x7a, 0x3c, 0x94, 0x9c, 0x30, 0xb0, 0xfe, 0x49, 0x81, 0xcb, 0x1b, 0xd8, 0xdf, 0x0e, 0xd2,
0xcc, 0x33, 0x94, 0x4e, 0x86, 0x8a, 0xe2, 0x6b, 0xae, 0xcc, 0x54, 0x6e, 0x9f, 0x89, 0xf8, 0x16,
0x98, 0x1f, 0x44, 0x1c, 0x72, 0x95, 0xd7, 0x02, 0x42, 0x78, 0xea, 0xe3, 0x3c, 0xd4, 0x3e, 0x16,
0xf5, 0x01, 0x4b, 0x23, 0x49, 0x39, 0x28, 0xe9, 0x72, 0x88, 0x94, 0x14, 0x69, 0x55, 0xc6, 0x06,
0xd4, 0x09, 0xc6, 0x87, 0x27, 0x49, 0x1a, 0x35, 0x4a, 0x28, 0x83, 0xfd, 0x26, 0xcc, 0x0d, 0x9d,
0x7d, 0x5a, 0xd6, 0x62, 0x43, 0xdc, 0x82, 0x57, 0x97, 0xd3, 0x23, 0xcf, 0x28, 0x21, 0xfa, 0x00,
0x66, 0x93, 0x7b, 0x15, 0x33, 0xed, 0x95, 0x24, 0x43, 0x1d, 0x68, 0x1a, 0x9e, 0x3b, 0x18, 0x60,
0xa3, 0x4b, 0x82, 0xad, 0x4a, 0xd9, 0xb6, 0x12, 0x74, 0xc1, 0x56, 0xea, 0x4f, 0x14, 0x98, 0xbf,
0xaf, 0xfb, 0xbd, 0x83, 0x35, 0x5b, 0x28, 0xe7, 0x14, 0xa6, 0xfd, 0x36, 0x54, 0x1e, 0x08, 0x45,
0x04, 0xf1, 0xeb, 0x4a, 0x0a, 0x43, 0x51, 0x95, 0x6b, 0x21, 0x85, 0xfa, 0x8d, 0x02, 0xe7, 0x58,
0x13, 0x11, 0x70, 0xf7, 0xdd, 0x3b, 0xd9, 0x94, 0x46, 0x02, 0xdd, 0x80, 0x86, 0xad, 0x7b, 0x87,
0x3b, 0x21, 0x4e, 0x91, 0xe1, 0x24, 0xa0, 0xea, 0x43, 0x00, 0xb1, 0xda, 0x22, 0xfd, 0x13, 0xf0,
0xff, 0x06, 0xcc, 0x88, 0x53, 0x85, 0xbf, 0x4d, 0x53, 0x6c, 0x80, 0xae, 0x7e, 0xab, 0x40, 0x23,
0x8c, 0xa0, 0xcc, 0xab, 0x1a, 0x90, 0x93, 0xbe, 0x94, 0xeb, 0xac, 0xa1, 0xb7, 0xa1, 0xc4, 0xdb,
0x46, 0xb1, 0xf7, 0xf5, 0xf8, 0xde, 0xa2, 0xa5, 0x8c, 0x84, 0x61, 0x06, 0xd0, 0x04, 0x11, 0x95,
0x91, 0x8c, 0x3a, 0xbc, 0xc3, 0xc8, 0x6b, 0x11, 0x08, 0xea, 0xc0, 0x6c, 0xbc, 0x68, 0x0b, 0x7c,
0x66, 0x71, 0x5c, 0xb4, 0x59, 0xd3, 0x7d, 0x9d, 0x05, 0x9b, 0x46, 0xac, 0x66, 0x23, 0xea, 0x7f,
0x8a, 0x50, 0x8d, 0xdc, 0x72, 0xe4, 0x26, 0x49, 0x95, 0xe6, 0xa6, 0xc7, 0xcd, 0xfc, 0x68, 0xe7,
0x70, 0x1d, 0x1a, 0x26, 0xcb, 0xd5, 0x5d, 0x61, 0x8a, 0x2c, 0xb8, 0x56, 0xb4, 0x3a, 0x87, 0x0a,
0xbf, 0x40, 0x0b, 0x50, 0x75, 0x86, 0x76, 0xd7, 0xdd, 0xef, 0x7a, 0xee, 0x11, 0x11, 0x2d, 0x48,
0xc5, 0x19, 0xda, 0xff, 0xbf, 0xaf, 0xb9, 0x47, 0x24, 0xac, 0x72, 0x4b, 0xc7, 0xac, 0x72, 0x17,
0xa0, 0x6a, 0xeb, 0x0f, 0xe9, 0xae, 0x5d, 0x67, 0x68, 0xb3, 0xee, 0x24, 0xaf, 0x55, 0x6c, 0xfd,
0xa1, 0xe6, 0x1e, 0xdd, 0x1b, 0xda, 0x68, 0x09, 0x9a, 0x96, 0x4e, 0xfc, 0x6e, 0xb4, 0xbd, 0x29,
0xb3, 0xf6, 0xa6, 0x41, 0xe1, 0xef, 0x87, 0x2d, 0xce, 0x68, 0xbd, 0x5c, 0x39, 0x45, 0xbd, 0x6c,
0xd8, 0x56, 0xb8, 0x11, 0x64, 0xaf, 0x97, 0x0d, 0xdb, 0x92, 0xdb, 0xbc, 0x01, 0x33, 0x7b, 0xac,
0x02, 0x22, 0xad, 0xea, 0xd8, 0x08, 0xb5, 0x4e, 0x8b, 0x1f, 0x5e, 0x28, 0x69, 0x01, 0x3a, 0x7a,
0x0b, 0x2a, 0x2c, 0xf5, 0x30, 0xda, 0x5a, 0x26, 0xda, 0x90, 0x80, 0x52, 0x1b, 0xd8, 0xf2, 0x75,
0x46, 0x5d, 0xcf, 0x46, 0x2d, 0x09, 0xd0, 0x2b, 0x70, 0xb6, 0xe7, 0x61, 0xdd, 0xc7, 0xc6, 0xca,
0xa3, 0x55, 0xd7, 0x1e, 0xe8, 0xcc, 0x98, 0x5a, 0x8d, 0x45, 0x65, 0xa9, 0xac, 0xa5, 0x7d, 0xa2,
0x81, 0xa1, 0x27, 0x57, 0xeb, 0x9e, 0x6b, 0xb7, 0x66, 0x79, 0x60, 0x88, 0x43, 0xd1, 0x65, 0x80,
0x20, 0x74, 0xeb, 0x7e, 0xab, 0xc9, 0xb4, 0x58, 0x11, 0x90, 0xf7, 0x7c, 0xf5, 0x0b, 0x38, 0x17,
0x5a, 0x48, 0x44, 0x1b, 0xa3, 0x8a, 0x55, 0x4e, 0xaa, 0xd8, 0xc9, 0xb5, 0xeb, 0x5f, 0x0a, 0x30,
0xbf, 0xa3, 0x3f, 0xc0, 0x4f, 0xbf, 0x4c, 0xce, 0x14, 0x8f, 0x37, 0x61, 0x8e, 0x55, 0xc6, 0xcb,
0x11, 0x7e, 0x26, 0x64, 0xe0, 0xa8, 0x3a, 0x47, 0x09, 0xd1, 0xbb, 0xb4, 0x74, 0xc0, 0xbd, 0xc3,
0x6d, 0xd7, 0x0c, 0xb3, 0xef, 0xe5, 0x94, 0x7d, 0x56, 0x25, 0x96, 0x16, 0xa5, 0x40, 0xdb, 0xa3,
0xa1, 0x8d, 0xe7, 0xdd, 0x17, 0x26, 0xf6, 0x5f, 0xa1, 0xf4, 0x93, 0x11, 0x0e, 0xb5, 0x60, 0x46,
0x64, 0x77, 0xe6, 0xf7, 0x65, 0x2d, 0x58, 0xa2, 0x6d, 0x38, 0xcb, 0x6f, 0xb0, 0x23, 0x8c, 0x9a,
0x5f, 0xbe, 0x9c, 0xe9, 0xf2, 0x69, 0xa4, 0x71, 0x9f, 0xa8, 0x1c, 0xd7, 0x27, 0x5a, 0x30, 0x23,
0xec, 0x94, 0xc5, 0x82, 0xb2, 0x16, 0x2c, 0x69, 0x13, 0x01, 0xa1, 0xc4, 0xa6, 0xcc, 0x02, 0xde,
0x81, 0xb2, 0xb4, 0xe1, 0x5c, 0x66, 0x1b, 0x96, 0x34, 0xc9, 0x28, 0x9c, 0x4f, 0x44, 0x61, 0xf5,
0x9f, 0x0a, 0xd4, 0xd6, 0x28, 0xd3, 0x9b, 0x6e, 0x9f, 0xe5, 0x8c, 0xeb, 0xd0, 0xf0, 0x70, 0xcf,
0xf5, 0x8c, 0x2e, 0x76, 0x7c, 0xcf, 0xc4, 0xbc, 0xdf, 0x2c, 0x68, 0x75, 0x0e, 0x7d, 0x9f, 0x03,
0x29, 0x1a, 0x0d, 0xac, 0xc4, 0xd7, 0xed, 0x41, 0x77, 0x9f, 0x3a, 0x70, 0x8e, 0xa3, 0x49, 0x28,
0xf3, 0xdf, 0xab, 0x50, 0x0b, 0xd1, 0x7c, 0x97, 0x9d, 0x5f, 0xd0, 0xaa, 0x12, 0xb6, 0xeb, 0xa2,
0xe7, 0xa1, 0xc1, 0xa4, 0xd6, 0xb5, 0xdc, 0x7e, 0x97, 0xf6, 0x66, 0x22, 0x9d, 0xd4, 0x0c, 0xc1,
0x16, 0xd5, 0x46, 0x1c, 0x8b, 0x98, 0x9f, 0x63, 0x91, 0x50, 0x24, 0xd6, 0x8e, 0xf9, 0x39, 0xa6,
0xd9, 0xbc, 0x4e, 0xb3, 0xe3, 0x3d, 0xd7, 0xc0, 0xbb, 0x27, 0xac, 0x25, 0x32, 0xcc, 0xe5, 0x2e,
0x41, 0x45, 0xde, 0x40, 0x5c, 0x29, 0x04, 0xa0, 0x75, 0x68, 0x04, 0x65, 0x66, 0x97, 0x77, 0x0f,
0x85, 0xb1, 0xb5, 0x5d, 0x24, 0xbf, 0x11, 0xad, 0x1e, 0x90, 0xb1, 0xa5, 0xba, 0x0e, 0xb5, 0xe8,
0x67, 0x7a, 0xea, 0x4e, 0xd2, 0x50, 0x24, 0x80, 0xda, 0xdb, 0xbd, 0xa1, 0x4d, 0x75, 0x2a, 0x42,
0x47, 0xb0, 0x54, 0xbf, 0x54, 0xa0, 0x2e, 0x92, 0xf2, 0x8e, 0x9c, 0x1b, 0xb3, 0xab, 0x29, 0xec,
0x6a, 0xec, 0x6f, 0xf4, 0xbf, 0xf1, 0xa1, 0xd3, 0xf3, 0xa9, 0x6e, 0xce, 0x36, 0x61, 0xf5, 0x6f,
0x2c, 0x23, 0x67, 0xe9, 0x56, 0x1f, 0x53, 0x43, 0x13, 0xaa, 0x61, 0x86, 0xd6, 0x82, 0x19, 0xdd,
0x30, 0x3c, 0x4c, 0x88, 0xe0, 0x23, 0x58, 0xd2, 0x2f, 0x0f, 0xb0, 0x47, 0x02, 0x93, 0xcf, 0x6b,
0xc1, 0x12, 0xbd, 0x05, 0x65, 0x59, 0x30, 0xe7, 0xd3, 0x8a, 0xa4, 0x28, 0x9f, 0xa2, 0xbb, 0x92,
0x14, 0xea, 0xd7, 0x39, 0x68, 0x08, 0x81, 0xad, 0x88, 0xac, 0x39, 0xd9, 0xf9, 0x56, 0xa0, 0xb6,
0x1f, 0x7a, 0xf7, 0xa4, 0x29, 0x4a, 0x34, 0x08, 0xc4, 0x68, 0xa6, 0x39, 0x60, 0x3c, 0x6f, 0x17,
0x4e, 0x95, 0xb7, 0x8b, 0xc7, 0x8c, 0x51, 0xea, 0x0f, 0xa0, 0x1a, 0xf9, 0xc2, 0x82, 0x2b, 0x9f,
0xab, 0x08, 0x51, 0x04, 0x4b, 0x74, 0x37, 0x2c, 0x4b, 0xb8, 0x0c, 0x2e, 0xa4, 0x1c, 0x92, 0xa8,
0x48, 0xd4, 0x3f, 0x28, 0x50, 0x12, 0x3b, 0x5f, 0x81, 0xaa, 0x88, 0x26, 0xac, 0x64, 0xe3, 0xbb,
0x83, 0x00, 0xd1, 0x9a, 0xed, 0xc9, 0x85, 0x93, 0x0b, 0x50, 0x4e, 0x04, 0x92, 0x19, 0x11, 0xd1,
0x83, 0x4f, 0x91, 0xe8, 0x41, 0x3f, 0xb1, 0xc0, 0xf1, 0x8d, 0xc2, 0x66, 0xc2, 0x1a, 0xee, 0xb9,
0x0f, 0xb0, 0xf7, 0xe8, 0xf4, 0x93, 0xb7, 0x37, 0x23, 0x96, 0x9a, 0xb1, 0xb5, 0x93, 0x04, 0xe8,
0xcd, 0x50, 0xdc, 0xf9, 0xb4, 0xc1, 0x43, 0x34, 0x74, 0x08, 0x3b, 0x0b, 0xc5, 0xfe, 0x73, 0x3e,
0x43, 0x8c, 0x5f, 0xe5, 0xa4, 0x25, 0xc9, 0x13, 0xe9, 0x18, 0xd4, 0x5f, 0x2a, 0x70, 0x61, 0x03,
0xfb, 0xeb, 0xf1, 0xbe, 0xfc, 0x59, 0x73, 0x65, 0x43, 0x3b, 0x8d, 0xa9, 0xd3, 0x68, 0xbd, 0x0d,
0x65, 0x39, 0x61, 0xe0, 0xd3, 0x5d, 0xb9, 0x56, 0x7f, 0xac, 0x40, 0x4b, 0x9c, 0xc2, 0xce, 0xa4,
0xd5, 0xb0, 0x85, 0x7d, 0x6c, 0x7c, 0xd7, 0x2d, 0xef, 0x3f, 0x14, 0x68, 0x46, 0x43, 0x39, 0x8b,
0xc6, 0xaf, 0x41, 0x91, 0x4d, 0x16, 0x04, 0x07, 0x53, 0x8d, 0x95, 0x63, 0xd3, 0x90, 0xc1, 0x2a,
0xb4, 0x5d, 0x99, 0x75, 0xc4, 0x32, 0xcc, 0x27, 0xf9, 0xe3, 0xe7, 0x13, 0x91, 0x5f, 0xdd, 0x21,
0xdd, 0x97, 0x4f, 0xee, 0x42, 0x80, 0xfa, 0x55, 0x0e, 0x5a, 0x61, 0x2b, 0xf1, 0x9d, 0x07, 0xf4,
0x31, 0x85, 0x66, 0xfe, 0x09, 0x15, 0x9a, 0x85, 0xe3, 0x06, 0xf1, 0xbf, 0xe7, 0xa0, 0x11, 0x8a,
0x63, 0xdb, 0xd2, 0x1d, 0x34, 0x0f, 0xa5, 0x81, 0xa5, 0x87, 0x13, 0x41, 0xb1, 0x42, 0x3b, 0xb2,
0x32, 0x89, 0x0b, 0xe0, 0xa5, 0x34, 0xe5, 0x8c, 0x91, 0xb0, 0x96, 0xd8, 0x82, 0xb6, 0x68, 0xbc,
0xc8, 0x67, 0x8d, 0xb6, 0xa8, 0x86, 0xb8, 0x15, 0xd0, 0x1e, 0xfb, 0x16, 0x20, 0xa1, 0xba, 0xae,
0xe9, 0x74, 0x09, 0xee, 0xb9, 0x8e, 0xc1, 0x95, 0x5a, 0xd4, 0x9a, 0xe2, 0x4b, 0xc7, 0xd9, 0xe1,
0x70, 0xf4, 0x1a, 0x14, 0xfc, 0x47, 0x03, 0x1e, 0x9e, 0x1b, 0xa9, 0x61, 0x2f, 0xe4, 0x6b, 0xf7,
0xd1, 0x00, 0x6b, 0x0c, 0x1d, 0x2d, 0x00, 0xd0, 0xad, 0x7c, 0x4f, 0x7f, 0x80, 0xad, 0xe0, 0xb7,
0xcc, 0x10, 0x42, 0xcd, 0x34, 0x98, 0x55, 0xcc, 0xf0, 0x9c, 0x20, 0x96, 0xea, 0x5f, 0x73, 0xd0,
0x0c, 0xb7, 0xd4, 0x30, 0x19, 0x5a, 0xfe, 0x58, 0xf9, 0x4d, 0x6e, 0xd0, 0xa6, 0x65, 0xfa, 0x77,
0xa1, 0x2a, 0xe6, 0x26, 0xc7, 0x50, 0x34, 0x70, 0x92, 0xcd, 0x09, 0x96, 0x57, 0x7c, 0x42, 0x96,
0x57, 0x3a, 0xae, 0xe5, 0xed, 0xc0, 0x7c, 0x10, 0xd0, 0x42, 0x84, 0x2d, 0xec, 0xeb, 0x13, 0x2a,
0x89, 0x2b, 0x50, 0xe5, 0x89, 0x8a, 0x67, 0x68, 0x5e, 0x5c, 0xc3, 0x9e, 0xec, 0x3a, 0xd5, 0x1f,
0xc2, 0x39, 0x16, 0x10, 0x92, 0xe3, 0xd5, 0x2c, 0xb3, 0x6e, 0x55, 0x96, 0xee, 0xb4, 0x4c, 0xe7,
0xd6, 0x5d, 0xd1, 0x62, 0x30, 0x75, 0x13, 0x9e, 0x4b, 0xec, 0x7f, 0x8a, 0x80, 0xaf, 0xfe, 0x4d,
0x81, 0x0b, 0x6b, 0x9e, 0x3b, 0xf8, 0xd8, 0xf4, 0xfc, 0xa1, 0x6e, 0xc5, 0x07, 0xf6, 0x4f, 0xa7,
0xf9, 0xf8, 0x20, 0x92, 0x63, 0x78, 0xd0, 0xb9, 0x95, 0xa2, 0xb2, 0x51, 0xa6, 0x84, 0xaa, 0x22,
0x19, 0xe9, 0x5f, 0xf9, 0x34, 0xe6, 0x05, 0xde, 0x94, 0x48, 0x9a, 0x25, 0x05, 0xa7, 0x4e, 0x23,
0xf2, 0x27, 0x9d, 0x46, 0x8c, 0xb1, 0xfe, 0xc2, 0x13, 0xb2, 0xfe, 0xe3, 0x16, 0xcf, 0xe8, 0x03,
0x88, 0x4f, 0x8a, 0x58, 0xd8, 0x39, 0xd1, 0x88, 0x69, 0x05, 0x20, 0x9c, 0x9a, 0x88, 0xd7, 0x16,
0x59, 0xb6, 0x89, 0x50, 0x51, 0x6d, 0xc9, 0x48, 0xc3, 0xa6, 0x9d, 0xb1, 0x2e, 0xff, 0x43, 0x68,
0xa7, 0x59, 0xe9, 0x69, 0x2c, 0xff, 0xeb, 0x1c, 0x40, 0xc7, 0x1e, 0xb8, 0x9e, 0xbf, 0xab, 0x93,
0xc3, 0x93, 0x95, 0x4b, 0xd7, 0xa0, 0x1e, 0x1a, 0x4c, 0xd7, 0x34, 0x52, 0xac, 0xc8, 0xa0, 0x2e,
0x21, 0xab, 0x36, 0x8a, 0x33, 0x52, 0xc9, 0x19, 0xe8, 0x22, 0x54, 0x3c, 0xf7, 0xa8, 0x4b, 0x3d,
0xc8, 0x60, 0xa9, 0xa5, 0xac, 0x95, 0x3d, 0xf7, 0x88, 0xfa, 0x95, 0x81, 0xce, 0xc3, 0x8c, 0xaf,
0x93, 0x43, 0x4a, 0xca, 0x8b, 0xfe, 0x12, 0x5d, 0x76, 0x0c, 0x74, 0x0e, 0x8a, 0xfb, 0xa6, 0x85,
0x79, 0xe0, 0xab, 0x68, 0x7c, 0x81, 0x5e, 0x0f, 0x7e, 0x32, 0x9f, 0xc9, 0xfc, 0x93, 0x1f, 0xff,
0xd5, 0xfc, 0x1b, 0x05, 0x66, 0x43, 0x81, 0xb0, 0x7a, 0x06, 0xbd, 0xc3, 0x9b, 0x3b, 0xbc, 0xea,
0x1a, 0x3c, 0x0a, 0x34, 0xc6, 0x4c, 0xf5, 0x39, 0x21, 0x2f, 0x82, 0x42, 0x92, 0x49, 0xf5, 0x24,
0xbd, 0x17, 0xbd, 0xb4, 0x69, 0x04, 0x3f, 0x2a, 0x94, 0x3c, 0xf7, 0xa8, 0x63, 0x90, 0x40, 0x1a,
0xfc, 0xe5, 0x10, 0xaf, 0x9e, 0xa8, 0x34, 0x56, 0xd9, 0xe3, 0xa1, 0x6b, 0x50, 0xc7, 0x9e, 0xe7,
0x7a, 0x5d, 0x1b, 0x13, 0xa2, 0xf7, 0x79, 0xa6, 0xad, 0x68, 0x35, 0x06, 0xdc, 0xe2, 0x30, 0xf5,
0xdb, 0x1c, 0x34, 0xc2, 0xab, 0x04, 0x3f, 0x25, 0x98, 0x46, 0xf0, 0x53, 0x82, 0x69, 0xd0, 0xac,
0xef, 0xf1, 0x28, 0x17, 0xea, 0xad, 0x22, 0x20, 0x1d, 0x83, 0x86, 0x79, 0xea, 0x3b, 0x8e, 0x6b,
0xe0, 0x50, 0x67, 0x10, 0x80, 0x3a, 0xc6, 0xa8, 0xea, 0x0b, 0x19, 0x54, 0x5f, 0x1c, 0x55, 0xfd,
0x3c, 0x94, 0xf6, 0x86, 0xbd, 0x43, 0xec, 0x33, 0xf7, 0xab, 0x68, 0x62, 0x15, 0x37, 0x89, 0x99,
0x84, 0x49, 0x48, 0xcd, 0x97, 0xa3, 0x9a, 0xbf, 0x08, 0x15, 0x3e, 0xaa, 0xee, 0xfa, 0x84, 0xfd,
0x10, 0x90, 0xd7, 0xca, 0x1c, 0xb0, 0x4b, 0xd0, 0x1b, 0x41, 0x39, 0x9b, 0x3a, 0xd8, 0x67, 0x71,
0x22, 0xa1, 0x7c, 0x51, 0xcc, 0xaa, 0x9f, 0x02, 0x0a, 0xbf, 0x9c, 0xae, 0xbd, 0x48, 0x48, 0x35,
0x97, 0x94, 0xaa, 0xfa, 0x47, 0x05, 0xe6, 0xa2, 0x87, 0x9d, 0x34, 0x0d, 0xbd, 0x03, 0x55, 0x93,
0x6d, 0xd3, 0xa5, 0xbe, 0x22, 0x1a, 0x8c, 0xcb, 0x13, 0xef, 0xac, 0x81, 0x19, 0x46, 0x83, 0x6b,
0x50, 0x3f, 0x72, 0xbd, 0x43, 0xd3, 0xe9, 0x77, 0x29, 0x67, 0x81, 0x85, 0xd6, 0x04, 0xf0, 0x1e,
0x85, 0xdd, 0xfc, 0x95, 0x02, 0x73, 0x23, 0x2d, 0x00, 0x6a, 0x00, 0x7c, 0xe4, 0xf4, 0x44, 0x6f,
0xd4, 0x3c, 0x83, 0x6a, 0x50, 0x0e, 0x3a, 0xa5, 0xa6, 0x82, 0xaa, 0x30, 0xb3, 0xeb, 0x32, 0xec,
0x66, 0x0e, 0x35, 0xa1, 0xc6, 0x09, 0x87, 0xbd, 0x1e, 0x26, 0xa4, 0x99, 0x97, 0x90, 0x75, 0xdd,
0xb4, 0x86, 0x1e, 0x6e, 0x16, 0x50, 0x1d, 0x2a, 0xbb, 0xae, 0x86, 0x2d, 0xac, 0x13, 0xdc, 0x2c,
0x22, 0x04, 0x0d, 0xb1, 0x08, 0x88, 0x4a, 0x11, 0x58, 0x40, 0x36, 0x73, 0x73, 0x3f, 0x5a, 0x53,
0xd3, 0x42, 0x13, 0x9d, 0x87, 0xb3, 0x1f, 0x39, 0x06, 0xde, 0x37, 0x1d, 0x6c, 0x84, 0x9f, 0x9a,
0x67, 0xd0, 0x59, 0x98, 0xed, 0x38, 0x0e, 0xf6, 0x22, 0x40, 0x85, 0x02, 0xb7, 0xb0, 0xd7, 0xc7,
0x11, 0x60, 0x0e, 0xcd, 0x41, 0x7d, 0xcb, 0x7c, 0x18, 0x01, 0xe5, 0x97, 0xff, 0x7d, 0x16, 0x2a,
0x6b, 0xba, 0xaf, 0xaf, 0xba, 0xae, 0x67, 0xa0, 0x01, 0x20, 0xf6, 0xa0, 0xc3, 0x1e, 0xb8, 0x8e,
0x7c, 0xf9, 0x84, 0x5e, 0x19, 0x93, 0x0a, 0x46, 0x51, 0x85, 0xc2, 0xdb, 0x37, 0xc6, 0x50, 0x24,
0xd0, 0xd5, 0x33, 0xc8, 0x66, 0x27, 0xd2, 0x42, 0x7d, 0xd7, 0xec, 0x1d, 0x06, 0x3f, 0xdd, 0x4d,
0x38, 0x31, 0x81, 0x1a, 0x9c, 0x98, 0x78, 0x50, 0x25, 0x16, 0xfc, 0xd5, 0x4d, 0x60, 0xf3, 0xea,
0x19, 0xf4, 0x19, 0x9c, 0xdb, 0xc0, 0x7e, 0xf8, 0xd0, 0x22, 0x38, 0x70, 0x79, 0xfc, 0x81, 0x23,
0xc8, 0xc7, 0x3c, 0x72, 0x13, 0x8a, 0xac, 0xdd, 0x46, 0x69, 0x2d, 0x6d, 0xf4, 0xf9, 0x6f, 0x7b,
0x71, 0x3c, 0x82, 0xdc, 0xed, 0x53, 0x98, 0x4d, 0x3c, 0x6f, 0x44, 0x2f, 0xa6, 0x90, 0xa5, 0x3f,
0x54, 0x6d, 0xdf, 0xcc, 0x82, 0x2a, 0xcf, 0xea, 0x43, 0x23, 0xfe, 0x1c, 0x04, 0x2d, 0xa5, 0xd0,
0xa7, 0x3e, 0x4d, 0x6b, 0xbf, 0x98, 0x01, 0x53, 0x1e, 0x64, 0x43, 0x33, 0xf9, 0xdc, 0x0e, 0xdd,
0x9c, 0xb8, 0x41, 0xdc, 0xdc, 0x5e, 0xca, 0x84, 0x2b, 0x8f, 0x7b, 0xc4, 0x8c, 0x60, 0xe4, 0xb9,
0x17, 0xba, 0x9d, 0xbe, 0xcd, 0xb8, 0x77, 0x68, 0xed, 0x3b, 0x99, 0xf1, 0xe5, 0xd1, 0x3f, 0xe2,
0x63, 0xbe, 0xb4, 0x27, 0x53, 0xe8, 0xd5, 0xf4, 0xed, 0x26, 0xbc, 0xf5, 0x6a, 0x2f, 0x1f, 0x87,
0x44, 0x32, 0xf1, 0x05, 0x9b, 0xcf, 0xa5, 0x3c, 0x3b, 0x4a, 0xfa, 0x5d, 0xb0, 0xdf, 0xf8, 0xf7,
0x54, 0xed, 0x57, 0x8f, 0x41, 0x21, 0x19, 0x70, 0x93, 0x0f, 0x1a, 0x03, 0x37, 0xbc, 0x33, 0xd5,
0x6a, 0x4e, 0xe6, 0x83, 0x9f, 0xc0, 0x6c, 0xe2, 0x47, 0xd2, 0x54, 0xaf, 0x49, 0xff, 0x21, 0xb5,
0x3d, 0x29, 0x35, 0x72, 0x97, 0x4c, 0x8c, 0x3b, 0xd1, 0x18, 0xeb, 0x4f, 0x19, 0x89, 0xb6, 0x6f,
0x66, 0x41, 0x95, 0x17, 0x21, 0x2c, 0x5c, 0x26, 0x46, 0x86, 0xe8, 0x56, 0xfa, 0x1e, 0xe9, 0xe3,
0xce, 0xf6, 0xcb, 0x19, 0xb1, 0xe5, 0xa1, 0x5d, 0x80, 0x0d, 0xec, 0x6f, 0x61, 0xdf, 0xa3, 0x36,
0x72, 0x23, 0x55, 0xe4, 0x21, 0x42, 0x70, 0xcc, 0x0b, 0x53, 0xf1, 0xe4, 0x01, 0xdf, 0x03, 0x14,
0xa4, 0xd8, 0xc8, 0x4f, 0xf4, 0xd7, 0x26, 0x0e, 0x5f, 0xf8, 0xa4, 0x64, 0x9a, 0x6e, 0x3e, 0x83,
0xe6, 0x96, 0xee, 0xd0, 0x96, 0x23, 0xdc, 0xf7, 0x56, 0x2a, 0x63, 0x49, 0xb4, 0x31, 0xd2, 0x1a,
0x8b, 0x2d, 0x2f, 0x73, 0x24, 0x73, 0xa8, 0x2e, 0x5d, 0x10, 0x27, 0x63, 0x4b, 0x28, 0x8d, 0x04,
0xe2, 0x98, 0xd8, 0x32, 0x01, 0x5f, 0x1e, 0xfc, 0x58, 0x61, 0xcf, 0x66, 0x13, 0x08, 0xf7, 0x4d,
0xff, 0x60, 0xdb, 0xd2, 0x1d, 0x92, 0x85, 0x05, 0x86, 0x78, 0x0c, 0x16, 0x04, 0xbe, 0x64, 0xc1,
0x80, 0x7a, 0x6c, 0xb6, 0x81, 0xd2, 0x7e, 0x67, 0x4f, 0x9b, 0xae, 0xb4, 0x97, 0xa6, 0x23, 0xca,
0x53, 0x0e, 0xa0, 0x1e, 0xd8, 0x2b, 0x17, 0xee, 0x8b, 0xe3, 0x38, 0x0d, 0x71, 0xc6, 0xb8, 0x5b,
0x3a, 0x6a, 0xd4, 0xdd, 0x46, 0xdb, 0x56, 0x94, 0x6d, 0xdc, 0x31, 0xc9, 0xdd, 0xc6, 0xf7, 0xc2,
0xea, 0x19, 0x74, 0x1f, 0x4a, 0xbc, 0xaa, 0x45, 0xcf, 0x4f, 0x2e, 0x78, 0xc5, 0x01, 0xd7, 0xa7,
0x60, 0x05, 0x1b, 0x2f, 0xff, 0xae, 0x08, 0xe5, 0xe0, 0x17, 0xd0, 0x67, 0x50, 0xea, 0x3d, 0x83,
0xda, 0xeb, 0x13, 0x98, 0x4d, 0x3c, 0x96, 0x4c, 0x0d, 0xcd, 0xe9, 0x0f, 0x2a, 0xa7, 0xc5, 0x96,
0xfb, 0xe2, 0x5f, 0xa8, 0x64, 0x18, 0x7e, 0x61, 0x5c, 0xfd, 0x96, 0x8c, 0xc0, 0x53, 0x36, 0x7e,
0xea, 0xf1, 0xf6, 0x1e, 0x40, 0x24, 0x1e, 0x4e, 0x1e, 0x72, 0x53, 0x17, 0x9f, 0xc6, 0xf0, 0xd6,
0x31, 0x2d, 0x76, 0xf2, 0x76, 0x2b, 0x77, 0xbf, 0xff, 0x6a, 0xdf, 0xf4, 0x0f, 0x86, 0x7b, 0xf4,
0xcb, 0x1d, 0x8e, 0xfa, 0xb2, 0xe9, 0x8a, 0xbf, 0xee, 0x04, 0x06, 0x72, 0x87, 0x51, 0xdf, 0xa1,
0x67, 0x0c, 0xf6, 0xf6, 0x4a, 0x6c, 0x75, 0xf7, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x9b,
0xc9, 0x27, 0xb3, 0x37, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -3732,7 +3791,7 @@ type DataCoordClient interface {
GetFlushState(ctx context.Context, in *milvuspb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error)
DropVirtualChannel(ctx context.Context, in *DropVirtualChannelRequest, opts ...grpc.CallOption) (*DropVirtualChannelResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(ctx context.Context, in *ImportTask, opts ...grpc.CallOption) (*ImportTaskResponse, error)
Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*ImportTaskResponse, error)
}
type dataCoordClient struct {
@ -3941,7 +4000,7 @@ func (c *dataCoordClient) DropVirtualChannel(ctx context.Context, in *DropVirtua
return out, nil
}
func (c *dataCoordClient) Import(ctx context.Context, in *ImportTask, opts ...grpc.CallOption) (*ImportTaskResponse, error) {
func (c *dataCoordClient) Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*ImportTaskResponse, error) {
out := new(ImportTaskResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/Import", in, out, opts...)
if err != nil {
@ -3976,7 +4035,7 @@ type DataCoordServer interface {
GetFlushState(context.Context, *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)
DropVirtualChannel(context.Context, *DropVirtualChannelRequest) (*DropVirtualChannelResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(context.Context, *ImportTask) (*ImportTaskResponse, error)
Import(context.Context, *ImportTaskRequest) (*ImportTaskResponse, error)
}
// UnimplementedDataCoordServer can be embedded to have forward compatible implementations.
@ -4049,7 +4108,7 @@ func (*UnimplementedDataCoordServer) GetFlushState(ctx context.Context, req *mil
func (*UnimplementedDataCoordServer) DropVirtualChannel(ctx context.Context, req *DropVirtualChannelRequest) (*DropVirtualChannelResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method DropVirtualChannel not implemented")
}
func (*UnimplementedDataCoordServer) Import(ctx context.Context, req *ImportTask) (*ImportTaskResponse, error) {
func (*UnimplementedDataCoordServer) Import(ctx context.Context, req *ImportTaskRequest) (*ImportTaskResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method Import not implemented")
}
@ -4454,7 +4513,7 @@ func _DataCoord_DropVirtualChannel_Handler(srv interface{}, ctx context.Context,
}
func _DataCoord_Import_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ImportTask)
in := new(ImportTaskRequest)
if err := dec(in); err != nil {
return nil, err
}
@ -4466,7 +4525,7 @@ func _DataCoord_Import_Handler(srv interface{}, ctx context.Context, dec func(in
FullMethod: "/milvus.proto.data.DataCoord/Import",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataCoordServer).Import(ctx, req.(*ImportTask))
return srv.(DataCoordServer).Import(ctx, req.(*ImportTaskRequest))
}
return interceptor(ctx, in, info, handler)
}
@ -4584,7 +4643,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)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(ctx context.Context, in *ImportTask, opts ...grpc.CallOption) (*commonpb.Status, error)
Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
}
type dataNodeClient struct {
@ -4649,7 +4708,7 @@ func (c *dataNodeClient) Compaction(ctx context.Context, in *CompactionPlan, opt
return out, nil
}
func (c *dataNodeClient) Import(ctx context.Context, in *ImportTask, opts ...grpc.CallOption) (*commonpb.Status, error) {
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...)
if err != nil {
@ -4668,7 +4727,7 @@ type DataNodeServer interface {
GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
Compaction(context.Context, *CompactionPlan) (*commonpb.Status, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
Import(context.Context, *ImportTask) (*commonpb.Status, error)
Import(context.Context, *ImportTaskRequest) (*commonpb.Status, error)
}
// UnimplementedDataNodeServer can be embedded to have forward compatible implementations.
@ -4693,7 +4752,7 @@ func (*UnimplementedDataNodeServer) GetMetrics(ctx context.Context, req *milvusp
func (*UnimplementedDataNodeServer) Compaction(ctx context.Context, req *CompactionPlan) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method Compaction not implemented")
}
func (*UnimplementedDataNodeServer) Import(ctx context.Context, req *ImportTask) (*commonpb.Status, error) {
func (*UnimplementedDataNodeServer) Import(ctx context.Context, req *ImportTaskRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method Import not implemented")
}
@ -4810,7 +4869,7 @@ func _DataNode_Compaction_Handler(srv interface{}, ctx context.Context, dec func
}
func _DataNode_Import_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ImportTask)
in := new(ImportTaskRequest)
if err := dec(in); err != nil {
return nil, err
}
@ -4822,7 +4881,7 @@ func _DataNode_Import_Handler(srv interface{}, ctx context.Context, dec func(int
FullMethod: "/milvus.proto.data.DataNode/Import",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataNodeServer).Import(ctx, req.(*ImportTask))
return srv.(DataNodeServer).Import(ctx, req.(*ImportTaskRequest))
}
return interceptor(ctx, in, info, handler)
}

View File

@ -206,7 +206,7 @@ func (coord *DataCoordMock) DropVirtualChannel(ctx context.Context, req *datapb.
return &datapb.DropVirtualChannelResponse{}, nil
}
func (coord *DataCoordMock) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
func (coord *DataCoordMock) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
return &datapb.ImportTaskResponse{}, nil
}

View File

@ -55,20 +55,22 @@ type importManager struct {
ctx context.Context // reserved
cancel context.CancelFunc // reserved
taskStore kv.MetaKv // Persistent task info storage.
busyNodes map[int64]bool // Set of all current working DataNodes.
// TODO: Make pendingTask a map to improve look up performance.
pendingTasks []*datapb.ImportTaskInfo // pending tasks
workingTasks map[int64]*datapb.ImportTaskInfo // in-progress tasks
pendingLock sync.RWMutex // lock pending task list
workingLock sync.RWMutex // lock working task map
nextTaskID int64 // for generating next import task ID
lastReqID int64 // for generating a unique ID for import request
pendingTasks []*datapb.ImportTaskInfo // pending tasks
workingTasks map[int64]*datapb.ImportTaskInfo // in-progress tasks
pendingLock sync.RWMutex // lock pending task list
workingLock sync.RWMutex // lock working task map
busyNodesLock sync.RWMutex // lock for working nodes.
nextTaskID int64 // for generating next import task ID
lastReqID int64 // for generating a unique ID for import request
callImportService func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse
callImportService func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse
}
// newImportManager helper function to create a importManager
func newImportManager(ctx context.Context, client kv.MetaKv, importService func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse) *importManager {
func newImportManager(ctx context.Context, client kv.MetaKv, importService func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse) *importManager {
ctx, cancel := context.WithCancel(ctx)
mgr := &importManager{
ctx: ctx,
@ -76,8 +78,10 @@ func newImportManager(ctx context.Context, client kv.MetaKv, importService func(
taskStore: client,
pendingTasks: make([]*datapb.ImportTaskInfo, 0, MaxPendingCount), // currently task queue max size is 32
workingTasks: make(map[int64]*datapb.ImportTaskInfo),
busyNodes: make(map[int64]bool),
pendingLock: sync.RWMutex{},
workingLock: sync.RWMutex{},
busyNodesLock: sync.RWMutex{},
nextTaskID: 0,
lastReqID: 0,
callImportService: importService,
@ -86,18 +90,20 @@ func newImportManager(ctx context.Context, client kv.MetaKv, importService func(
return mgr
}
func (m *importManager) init() error {
func (m *importManager) init(ctx context.Context) error {
// Read tasks from etcd and save them as pendingTasks or workingTasks.
m.load()
m.sendOutTasks()
m.sendOutTasks(ctx)
return nil
}
// sendOutTasks pushes all pending tasks to DataCoord, gets DataCoord response and re-add these tasks as working tasks.
func (m *importManager) sendOutTasks() error {
func (m *importManager) sendOutTasks(ctx context.Context) error {
m.pendingLock.Lock()
m.busyNodesLock.Lock()
defer m.pendingLock.Unlock()
defer m.busyNodesLock.Unlock()
// Trigger Import() action to DataCoord.
for len(m.pendingTasks) > 0 {
@ -117,8 +123,17 @@ func (m *importManager) sendOutTasks() error {
}
log.Debug("sending import task to DataCoord", zap.Int64("taskID", task.GetId()))
// Get all busy dataNodes for reference.
var busyNodeList []int64
for k := range m.busyNodes {
busyNodeList = append(busyNodeList, k)
}
// Call DataCoord.Import().
resp := m.callImportService(m.ctx, it)
resp := m.callImportService(ctx, &datapb.ImportTaskRequest{
ImportTask: it,
WorkingNodes: busyNodeList,
})
if resp.Status.ErrorCode == commonpb.ErrorCode_UnexpectedError {
log.Debug("import task is rejected", zap.Int64("task ID", it.GetTaskId()))
break
@ -127,6 +142,8 @@ func (m *importManager) sendOutTasks() error {
log.Debug("import task successfully assigned to DataNode",
zap.Int64("task ID", it.GetTaskId()),
zap.Int64("DataNode ID", task.GetDatanodeId()))
// Add new working dataNode to busyNodes.
m.busyNodes[resp.GetDatanodeId()] = true
// erase this task from head of pending list if the callImportService succeed
m.pendingTasks = m.pendingTasks[1:]
@ -163,7 +180,7 @@ func (m *importManager) genReqID() int64 {
// importJob processes the import request, generates import tasks, sends these tasks to DataCoord, and returns
// immediately.
func (m *importManager) importJob(req *milvuspb.ImportRequest, cID int64) *milvuspb.ImportResponse {
func (m *importManager) importJob(ctx context.Context, req *milvuspb.ImportRequest, cID int64) *milvuspb.ImportResponse {
if req == nil || len(req.Files) == 0 {
return &milvuspb.ImportResponse{
Status: &commonpb.Status{
@ -267,7 +284,7 @@ func (m *importManager) importJob(req *milvuspb.ImportRequest, cID int64) *milvu
log.Info("column-based import request processed", zap.Int64("reqID", reqID), zap.Int64("taskID", newTask.GetId()))
}
}()
m.sendOutTasks()
m.sendOutTasks(ctx)
return resp
}

View File

@ -56,7 +56,7 @@ func TestImportManager_NewImportManager(t *testing.T) {
mockKv.SaveWithLease(BuildImportTaskKey(1), "value", 1)
mockKv.SaveWithLease(BuildImportTaskKey(2), string(taskInfo1), 2)
mockKv.SaveWithLease(BuildImportTaskKey(3), string(taskInfo2), 3)
fn := func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
fn := func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -65,7 +65,7 @@ func TestImportManager_NewImportManager(t *testing.T) {
}
mgr := newImportManager(context.TODO(), mockKv, fn)
assert.NotNil(t, mgr)
mgr.init()
mgr.init(context.TODO())
}
func TestImportManager_ImportJob(t *testing.T) {
@ -74,7 +74,7 @@ func TestImportManager_ImportJob(t *testing.T) {
mockKv := &kv.MockMetaKV{}
mockKv.InMemKv = make(map[string]string)
mgr := newImportManager(context.TODO(), mockKv, nil)
resp := mgr.importJob(nil, colID)
resp := mgr.importJob(context.TODO(), nil, colID)
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
rowReq := &milvuspb.ImportRequest{
@ -84,7 +84,7 @@ func TestImportManager_ImportJob(t *testing.T) {
Files: []string{"f1", "f2", "f3"},
}
resp = mgr.importJob(rowReq, colID)
resp = mgr.importJob(context.TODO(), rowReq, colID)
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
colReq := &milvuspb.ImportRequest{
@ -100,7 +100,7 @@ func TestImportManager_ImportJob(t *testing.T) {
},
}
fn := func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
fn := func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -109,16 +109,16 @@ func TestImportManager_ImportJob(t *testing.T) {
}
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(rowReq, colID)
resp = mgr.importJob(context.TODO(), rowReq, colID)
assert.Equal(t, len(rowReq.Files), len(mgr.pendingTasks))
assert.Equal(t, 0, len(mgr.workingTasks))
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(colReq, colID)
resp = mgr.importJob(context.TODO(), colReq, colID)
assert.Equal(t, 1, len(mgr.pendingTasks))
assert.Equal(t, 0, len(mgr.workingTasks))
fn = func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
fn = func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -127,17 +127,17 @@ func TestImportManager_ImportJob(t *testing.T) {
}
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(rowReq, colID)
resp = mgr.importJob(context.TODO(), rowReq, colID)
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, len(rowReq.Files), len(mgr.workingTasks))
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(colReq, colID)
resp = mgr.importJob(context.TODO(), colReq, colID)
assert.Equal(t, 0, len(mgr.pendingTasks))
assert.Equal(t, 1, len(mgr.workingTasks))
count := 0
fn = func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
fn = func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
if count >= 2 {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
@ -154,7 +154,7 @@ func TestImportManager_ImportJob(t *testing.T) {
}
mgr = newImportManager(context.TODO(), mockKv, fn)
resp = mgr.importJob(rowReq, colID)
resp = mgr.importJob(context.TODO(), rowReq, colID)
assert.Equal(t, len(rowReq.Files)-2, len(mgr.pendingTasks))
assert.Equal(t, 2, len(mgr.workingTasks))
}
@ -164,7 +164,7 @@ func TestImportManager_TaskState(t *testing.T) {
colID := int64(100)
mockKv := &kv.MockMetaKV{}
mockKv.InMemKv = make(map[string]string)
fn := func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
fn := func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -180,7 +180,7 @@ func TestImportManager_TaskState(t *testing.T) {
}
mgr := newImportManager(context.TODO(), mockKv, fn)
mgr.importJob(rowReq, colID)
mgr.importJob(context.TODO(), rowReq, colID)
state := &rootcoordpb.ImportResult{
TaskId: 10000,

View File

@ -152,7 +152,7 @@ type Core struct {
CallWatchChannels func(ctx context.Context, collectionID int64, channelNames []string) error
//assign import task to data service
CallImportService func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse
CallImportService func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse
//Proxy manager
proxyManager *proxyManager
@ -738,7 +738,7 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
}
return nil
}
c.CallImportService = func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
c.CallImportService = func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
resp := &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -1132,7 +1132,7 @@ func (c *Core) Init() error {
c.impTaskKv,
c.CallImportService,
)
c.importManager.init()
c.importManager.init(c.ctx)
})
if initError != nil {
log.Debug("RootCoord init error", zap.Error(initError))
@ -2256,7 +2256,7 @@ func (c *Core) Import(ctx context.Context, req *milvuspb.ImportRequest) (*milvus
zap.String("partition name", req.GetPartitionName()),
zap.Int("# of files = ", len(req.GetFiles())),
)
resp := c.importManager.importJob(req, cID)
resp := c.importManager.importJob(ctx, req, cID)
return resp, nil
}
@ -2280,14 +2280,14 @@ func (c *Core) GetImportState(ctx context.Context, req *milvuspb.GetImportStateR
}
// ReportImport reports import task state to RootCoord.
func (c *Core) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) (*commonpb.Status, error) {
func (c *Core) ReportImport(ctx context.Context, ir *rootcoordpb.ImportResult) (*commonpb.Status, error) {
if code, ok := c.checkHealthy(); !ok {
return failStatus(commonpb.ErrorCode_UnexpectedError, "StateCode="+internalpb.StateCode_name[int32(code)]), nil
}
log.Info("receive import state report")
log.Info("receive import state report", zap.Any("import result", ir.String()))
// Upon receiving ReportImport request, update the related task's state in task store.
ti, err := c.importManager.updateTaskState(req)
ti, err := c.importManager.updateTaskState(ir)
if err != nil {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UpdateImportTaskFailure,
@ -2311,7 +2311,19 @@ func (c *Core) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult)
// Start a loop to check segments' index states periodically.
c.wg.Add(1)
go c.CheckCompleteIndexLoop(ctx, ti, colName, req.Segments)
go c.checkCompleteIndexLoop(ctx, ti, colName, ir.Segments)
// When DataNode has done its thing, remove it from the busy node list.
c.importManager.busyNodesLock.Lock()
defer c.importManager.busyNodesLock.Unlock()
delete(c.importManager.busyNodes, ir.GetDatanodeId())
log.Info("dataNode is no longer busy",
zap.Int64("dataNode ID", ir.GetDatanodeId()),
zap.Int64("task ID", ir.GetTaskId()))
// Start a loop to check segments' index states periodically.
c.wg.Add(1)
go c.checkCompleteIndexLoop(ctx, ti, colName, ir.Segments)
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -2407,10 +2419,10 @@ func (c *Core) CountCompleteIndex(ctx context.Context, collectionName string, co
return ct, nil
}
// CheckCompleteIndexLoop checks index build states for an import task's segments and bring these segments online when
// the criteria are met. CheckCompleteIndexLoop does the check every CheckCompleteIndexInterval and exits if:
// checkCompleteIndexLoop checks index build states for an import task's segments and bring these segments online when
// the criteria are met. checkCompleteIndexLoop does the check every CheckCompleteIndexInterval and exits if:
// (1) a certain percent of indices are built, (2) when context is done or (3) when the task is expired.
func (c *Core) CheckCompleteIndexLoop(ctx context.Context, ti *datapb.ImportTaskInfo, colName string, segIDs []UniqueID) {
func (c *Core) checkCompleteIndexLoop(ctx context.Context, ti *datapb.ImportTaskInfo, colName string, segIDs []UniqueID) {
defer c.wg.Done()
ticker := time.NewTicker(CheckCompleteIndexInterval)
spent := time.Duration(time.Unix(time.Now().Unix()-ti.GetCreateTs(), 0).Nanosecond())
@ -2422,7 +2434,7 @@ func (c *Core) CheckCompleteIndexLoop(ctx context.Context, ti *datapb.ImportTask
for {
select {
case <-c.ctx.Done():
log.Info("(in loop)context done, exiting CheckCompleteIndexLoop", zap.Int64("task ID", ti.GetId()))
log.Info("(in loop)context done, exiting checkCompleteIndexLoop", zap.Int64("task ID", ti.GetId()))
return
case <-ticker.C:
log.Info("(in loop)check segments' index states", zap.Int64("task ID", ti.GetId()))

View File

@ -27,13 +27,12 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/kv"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -174,7 +173,7 @@ func (d *dataMock) WatchChannels(ctx context.Context, req *datapb.WatchChannelsR
}}, nil
}
func (d *dataMock) Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error) {
func (d *dataMock) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -2812,7 +2811,7 @@ func TestCheckInit(t *testing.T) {
c.CallWatchChannels = func(ctx context.Context, collectionID int64, channelNames []string) error {
return nil
}
c.CallImportService = func(ctx context.Context, req *datapb.ImportTask) *datapb.ImportTaskResponse {
c.CallImportService = func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
return &datapb.ImportTaskResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,

View File

@ -76,7 +76,7 @@ type DataNode interface {
//
// Return status indicates if this operation is processed successfully or fail cause;
// error is always nil
Import(ctx context.Context, req *datapb.ImportTask) (*commonpb.Status, error)
Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error)
}
// DataNodeComponent is used by grpc server of DataNode
@ -274,7 +274,7 @@ type DataCoord interface {
// The `Status` in response struct `ImportResponse` indicates if this operation is processed successfully or fail cause;
// the `tasks` in `ImportResponse` return an id list of tasks.
// error is always nil
Import(ctx context.Context, req *datapb.ImportTask) (*datapb.ImportTaskResponse, error)
Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
}
// DataCoordComponent defines the interface of DataCoord component.

View File

@ -119,6 +119,6 @@ func (m *DataCoordClient) DropVirtualChannel(ctx context.Context, req *datapb.Dr
return &datapb.DropVirtualChannelResponse{}, m.Err
}
func (m *DataCoordClient) Import(ctx context.Context, req *datapb.ImportTask, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
func (m *DataCoordClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
return &datapb.ImportTaskResponse{}, m.Err
}

View File

@ -55,6 +55,6 @@ func (m *DataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionP
return &commonpb.Status{}, m.Err
}
func (m *DataNodeClient) Import(ctx context.Context, req *datapb.ImportTask, opts ...grpc.CallOption) (*commonpb.Status, error) {
func (m *DataNodeClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}