From f5e63177d24cfffb5cc54bffd06d7bcbd574fcf9 Mon Sep 17 00:00:00 2001 From: Ten Thousand Leaves <69466447+soothing-rain@users.noreply.github.com> Date: Thu, 2 Jun 2022 18:54:04 +0800 Subject: [PATCH] Fix an issue with bulk load with multiple DataNodes (#17324) issue: #17300 Signed-off-by: Yuchen Gao --- .../cluster/docker-compose-apple-silicon.yml | 187 ++++++ deployments/docker/cluster/docker-compose.yml | 42 +- internal/datacoord/channel_manager.go | 11 + internal/datacoord/cluster.go | 5 + internal/datacoord/cluster_test.go | 72 +++ internal/datacoord/mock_test.go | 4 + internal/datacoord/server_test.go | 53 +- internal/datacoord/services.go | 51 +- internal/datacoord/session_manager.go | 28 +- internal/datanode/data_node.go | 110 ++-- internal/datanode/data_node_test.go | 69 ++ internal/datanode/meta_service.go | 27 + internal/datanode/mock_test.go | 7 + internal/datanode/segment_replica.go | 2 +- .../distributed/datacoord/client/client.go | 14 + .../datacoord/client/client_test.go | 3 + internal/distributed/datacoord/service.go | 4 + .../distributed/datacoord/service_test.go | 16 + .../distributed/datanode/client/client.go | 14 + .../datanode/client/client_test.go | 3 + internal/distributed/datanode/service.go | 4 + internal/distributed/datanode/service_test.go | 15 +- internal/distributed/proxy/service_test.go | 4 + internal/proto/data_coord.proto | 13 + internal/proto/datapb/data_coord.pb.go | 603 +++++++++++------- internal/proxy/data_coord_mock_test.go | 4 + internal/querynode/segment_loader_test.go | 5 +- internal/types/types.go | 7 + internal/util/mock/grpc_datacoord_client.go | 4 + internal/util/mock/grpc_datanode_client.go | 4 + 30 files changed, 1101 insertions(+), 284 deletions(-) create mode 100644 deployments/docker/cluster/docker-compose-apple-silicon.yml diff --git a/deployments/docker/cluster/docker-compose-apple-silicon.yml b/deployments/docker/cluster/docker-compose-apple-silicon.yml new file mode 100644 index 0000000000..a07feaa0f6 --- /dev/null +++ b/deployments/docker/cluster/docker-compose-apple-silicon.yml @@ -0,0 +1,187 @@ +version: '3.5' + +services: + etcd: + container_name: milvus-etcd + image: quay.io/coreos/etcd:v3.5.0 + environment: + - ETCD_AUTO_COMPACTION_MODE=revision + - ETCD_AUTO_COMPACTION_RETENTION=1000 + - ETCD_QUOTA_BACKEND_BYTES=4294967296 + volumes: + - ${DOCKER_VOLUME_DIRECTORY:-.}/volumes/etcd:/etcd + command: etcd -advertise-client-urls=http://127.0.0.1:2379 -listen-client-urls http://0.0.0.0:2379 --data-dir /etcd + + pulsar: + container_name: milvus-pulsar + image: milvusdb/pulsar:v2.8.2-m1 + volumes: + - ${DOCKER_VOLUME_DIRECTORY:-.}/volumes/pulsar:/pulsar/data + environment: + # bin/apply-config-from-env.py script will modify the configuration file based on the environment variables + # nettyMaxFrameSizeBytes must be calculated from maxMessageSize + 10240 (padding) + - nettyMaxFrameSizeBytes=104867840 # this is 104857600 + 10240 (padding) + - defaultRetentionTimeInMinutes=10080 + - defaultRetentionSizeInMB=8192 + # maxMessageSize is missing from standalone.conf, must use PULSAR_PREFIX_ to get it configured + - PULSAR_PREFIX_maxMessageSize=104857600 + - PULSAR_GC=-XX:+UseG1GC + + minio: + container_name: milvus-minio + image: minio/minio:RELEASE.2022-03-17T06-34-49Z + ports: + - "9000:9000" + - "9001:9001" + environment: + MINIO_ACCESS_KEY: minioadmin + MINIO_SECRET_KEY: minioadmin + volumes: + - ${DOCKER_VOLUME_DIRECTORY:-.}/volumes/minio:/minio_data + command: minio server /minio_data --console-address ":9001" + healthcheck: + test: ["CMD", "curl", "-f", "http://localhost:9000/minio/health/live"] + interval: 30s + timeout: 20s + retries: 3 + + rootcoord: + container_name: milvus-rootcoord + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "rootcoord"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + ROOT_COORD_ADDRESS: rootcoord:53100 + depends_on: + - "etcd" + - "pulsar" + - "minio" + + proxy: + container_name: milvus-proxy + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "proxy"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + ports: + - "19530:19530" + + querycoord: + container_name: milvus-querycoord + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "querycoord"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + QUERY_COORD_ADDRESS: querycoord:19531 + depends_on: + - "etcd" + - "pulsar" + - "minio" + + querynode: + container_name: milvus-querynode + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "querynode"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + depends_on: + - "querycoord" + + indexcoord: + container_name: milvus-indexcoord + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "indexcoord"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + INDEX_COORD_ADDRESS: indexcoord:31000 + depends_on: + - "etcd" + - "pulsar" + - "minio" + + indexnode: + container_name: milvus-indexnode + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "indexnode"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + INDEX_COORD_ADDRESS: indexcoord:31000 + depends_on: + - "indexcoord" + + datacoord: + container_name: milvus-datacoord + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "datacoord"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + DATA_COORD_ADDRESS: datacoord:13333 + depends_on: + - "etcd" + - "pulsar" + - "minio" + + datanode: + container_name: milvus-datanode + image: milvusdb/milvus:v2.0.2 + command: ["milvus", "run", "datanode"] + environment: + ETCD_ENDPOINTS: etcd:2379 + MINIO_ADDRESS: minio:9000 + PULSAR_ADDRESS: pulsar://pulsar:6650 + depends_on: + - "datacoord" + +# ---------------------------------------------------------------------------------------- +# If you need multiple instances of a service, say DataNode, use the following script: +# ---------------------------------------------------------------------------------------- +# datanode-1: +# container_name: milvus-datanode-1 +# image: milvusdb/milvus:v2.0.2 +# command: ["milvus", "run", "datanode"] +# environment: +# ETCD_ENDPOINTS: etcd:2379 +# MINIO_ADDRESS: minio:9000 +# PULSAR_ADDRESS: pulsar://pulsar:6650 +# depends_on: +# - "datacoord" +# +# datanode-2: +# container_name: milvus-datanode-2 +# image: milvusdb/milvus:v2.0.2 +# command: ["milvus", "run", "datanode"] +# environment: +# ETCD_ENDPOINTS: etcd:2379 +# MINIO_ADDRESS: minio:9000 +# PULSAR_ADDRESS: pulsar://pulsar:6650 +# depends_on: +# - "datacoord" +# +# datanode-3: +# container_name: milvus-datanode-3 +# image: milvusdb/milvus:v2.0.2 +# command: ["milvus", "run", "datanode"] +# environment: +# ETCD_ENDPOINTS: etcd:2379 +# MINIO_ADDRESS: minio:9000 +# PULSAR_ADDRESS: pulsar://pulsar:6650 +# depends_on: +# - "datacoord" + +networks: + default: + name: milvus diff --git a/deployments/docker/cluster/docker-compose.yml b/deployments/docker/cluster/docker-compose.yml index 8dc8dce262..a5c7606506 100644 --- a/deployments/docker/cluster/docker-compose.yml +++ b/deployments/docker/cluster/docker-compose.yml @@ -37,9 +37,12 @@ services: environment: MINIO_ACCESS_KEY: minioadmin MINIO_SECRET_KEY: minioadmin + ports: + - "9000:9000" + - "9001:9001" volumes: - ${DOCKER_VOLUME_DIRECTORY:-.}/volumes/minio:/minio_data - command: minio server /minio_data + command: minio server /minio_data --console-address ":9001" healthcheck: test: ["CMD", "curl", "-f", "http://localhost:9000/minio/health/live"] interval: 30s @@ -147,6 +150,43 @@ services: depends_on: - "datacoord" +# ---------------------------------------------------------------------------------------- +# If you need multiple instances of a service, say DataNode, use the following script: +# ---------------------------------------------------------------------------------------- +# datanode-1: +# container_name: milvus-datanode-1 +# image: milvusdb/milvus:v2.0.2 +# command: ["milvus", "run", "datanode"] +# environment: +# ETCD_ENDPOINTS: etcd:2379 +# MINIO_ADDRESS: minio:9000 +# PULSAR_ADDRESS: pulsar://pulsar:6650 +# depends_on: +# - "datacoord" +# +# datanode-2: +# container_name: milvus-datanode-2 +# image: milvusdb/milvus:v2.0.2 +# command: ["milvus", "run", "datanode"] +# environment: +# ETCD_ENDPOINTS: etcd:2379 +# MINIO_ADDRESS: minio:9000 +# PULSAR_ADDRESS: pulsar://pulsar:6650 +# depends_on: +# - "datacoord" +# +# datanode-3: +# container_name: milvus-datanode-3 +# image: milvusdb/milvus:v2.0.2 +# command: ["milvus", "run", "datanode"] +# environment: +# ETCD_ENDPOINTS: etcd:2379 +# MINIO_ADDRESS: minio:9000 +# PULSAR_ADDRESS: pulsar://pulsar:6650 +# depends_on: +# - "datacoord" + + networks: default: name: milvus diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index 7b474d887e..92755c639c 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -803,6 +803,17 @@ func (c *ChannelManager) getChannelByNodeAndName(nodeID UniqueID, channelName st return ret } +func (c *ChannelManager) getNodeIDByChannelName(chName string) (bool, UniqueID) { + for _, nodeChannel := range c.GetChannels() { + for _, ch := range nodeChannel.Channels { + if ch.Name == chName { + return true, nodeChannel.NodeID + } + } + } + return false, 0 +} + func (c *ChannelManager) isMarkedDrop(channelName string) bool { return c.h.CheckShouldDropChannel(channelName) } diff --git a/internal/datacoord/cluster.go b/internal/datacoord/cluster.go index a2f1779386..973f792187 100644 --- a/internal/datacoord/cluster.go +++ b/internal/datacoord/cluster.go @@ -149,6 +149,11 @@ func (c *Cluster) ReCollectSegmentStats(ctx context.Context, nodeID int64) { c.sessionManager.ReCollectSegmentStats(ctx, nodeID) } +// AddSegment triggers a AddSegment call from session manager. +func (c *Cluster) AddSegment(ctx context.Context, nodeID int64, req *datapb.AddSegmentRequest) { + c.sessionManager.AddSegment(ctx, nodeID, req) +} + // GetSessions returns all sessions func (c *Cluster) GetSessions() []*Session { return c.sessionManager.GetSessions() diff --git a/internal/datacoord/cluster_test.go b/internal/datacoord/cluster_test.go index 9df7fe39ed..24c607ef14 100644 --- a/internal/datacoord/cluster_test.go +++ b/internal/datacoord/cluster_test.go @@ -25,6 +25,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/milvus-io/milvus/internal/kv" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" + "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" "github.com/stretchr/testify/assert" @@ -640,3 +641,74 @@ func TestCluster_ReCollectSegmentStats(t *testing.T) { time.Sleep(500 * time.Millisecond) }) } + +func TestCluster_AddSegment(t *testing.T) { + kv := getMetaKv(t) + defer func() { + kv.RemoveWithPrefix("") + kv.Close() + }() + + t.Run("add segment succeed", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.TODO()) + defer cancel() + var mockSessionCreator = func(ctx context.Context, addr string) (types.DataNode, error) { + return newMockDataNodeClient(1, nil) + } + sessionManager := NewSessionManager(withSessionCreator(mockSessionCreator)) + 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.AddSegment(ctx, 1, &datapb.AddSegmentRequest{ + Base: &commonpb.MsgBase{ + SourceID: 0, + }, + }) + }) + time.Sleep(500 * time.Millisecond) + }) + + t.Run("add segment failed", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.TODO()) + 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.AddSegment(ctx, 1, &datapb.AddSegmentRequest{ + Base: &commonpb.MsgBase{ + SourceID: 0, + }, + }) + }) + time.Sleep(500 * time.Millisecond) + }) +} diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index 4b8d899e6d..5e6a2e5952 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -209,6 +209,10 @@ func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRe return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil } +func (c *mockDataNodeClient) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*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/datacoord/server_test.go b/internal/datacoord/server_test.go index ff904ddd9a..7446d7f5af 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -2520,7 +2520,7 @@ func TestDataCoordServer_SetSegmentState(t *testing.T) { }) } -func TestImport(t *testing.T) { +func TestDataCoord_Import(t *testing.T) { t.Run("normal case", func(t *testing.T) { svr := newTestServer(t, nil) defer closeTestServer(t, svr) @@ -2645,6 +2645,57 @@ func TestImport(t *testing.T) { }) } +func TestDataCoord_AddSegment(t *testing.T) { + t.Run("test add segment", func(t *testing.T) { + svr := newTestServer(t, nil) + defer closeTestServer(t, svr) + + err := svr.channelManager.AddNode(110) + assert.Nil(t, err) + err = svr.channelManager.Watch(&channel{"ch1", 100}) + assert.Nil(t, err) + + status, err := svr.AddSegment(context.TODO(), &datapb.AddSegmentRequest{ + SegmentId: 100, + ChannelName: "ch1", + CollectionId: 100, + PartitionId: 100, + RowNum: int64(1), + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode()) + }) + + t.Run("test add segment w/ bad channel name", func(t *testing.T) { + svr := newTestServer(t, nil) + defer closeTestServer(t, svr) + + err := svr.channelManager.AddNode(110) + assert.Nil(t, err) + err = svr.channelManager.Watch(&channel{"ch1", 100}) + assert.Nil(t, err) + + status, err := svr.AddSegment(context.TODO(), &datapb.AddSegmentRequest{ + SegmentId: 100, + ChannelName: "non-channel", + CollectionId: 100, + PartitionId: 100, + RowNum: int64(1), + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode()) + }) + + t.Run("test add segment w/ closed server", func(t *testing.T) { + svr := newTestServer(t, nil) + closeTestServer(t, svr) + + status, err := svr.AddSegment(context.TODO(), &datapb.AddSegmentRequest{}) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode()) + }) +} + // https://github.com/milvus-io/milvus/issues/15659 func TestIssue15659(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index ec44004823..b360772fc5 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -331,12 +331,15 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath return resp, nil } - channel := segment.GetInsertChannel() - if !s.channelManager.Match(nodeID, channel) { - FailResponse(resp, fmt.Sprintf("channel %s is not watched on node %d", channel, nodeID)) - resp.ErrorCode = commonpb.ErrorCode_MetaFailed - log.Warn("node is not matched with channel", zap.String("channel", channel), zap.Int64("nodeID", nodeID)) - return resp, nil + // No need to check import channel--node matching in data import case. + if !req.GetImporting() { + channel := segment.GetInsertChannel() + if !s.channelManager.Match(nodeID, channel) { + FailResponse(resp, fmt.Sprintf("channel %s is not watched on node %d", channel, nodeID)) + resp.ErrorCode = commonpb.ErrorCode_MetaFailed + log.Warn("node is not matched with channel", zap.String("channel", channel), zap.Int64("nodeID", nodeID)) + return resp, nil + } } if req.GetDropped() { @@ -1004,7 +1007,7 @@ func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateR // Import distributes the import tasks to dataNodes. // It returns a failed status if no dataNode is available or if any error occurs. func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) { - log.Info("DataCoord receive import request", zap.Any("import task request", itr)) + log.Info("DataCoord receives import request", zap.Any("import task request", itr)) resp := &datapb.ImportTaskResponse{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_UnexpectedError, @@ -1012,14 +1015,14 @@ func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*da } if s.isClosed() { - log.Error("failed to import for closed dataCoord service") + log.Error("failed to import for closed DataCoord service") resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID()) return resp, nil } nodes := s.channelManager.store.GetNodes() if len(nodes) == 0 { - log.Error("import failed as all dataNodes are offline") + log.Error("import failed as all DataNodes are offline") return resp, nil } @@ -1033,7 +1036,7 @@ func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*da s.cluster.Import(s.ctx, resp.GetDatanodeId(), itr) } else { // No dataNode is available, reject the import request. - msg := "all dataNodes are busy working on data import, the task has been rejected and wait for idle datanode" + msg := "all DataNodes are busy working on data import, the task has been rejected and wait for idle datanode" log.Info(msg, zap.Int64("task ID", itr.GetImportTask().GetTaskId())) resp.Status.Reason = msg return resp, nil @@ -1137,3 +1140,31 @@ func (s *Server) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegm resp.ErrorCode = commonpb.ErrorCode_Success return resp, nil } + +func (s *Server) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + log.Info("DataCoord putting segment to the right DataNode", + zap.Int64("segment ID", req.GetSegmentId()), + zap.Int64("collection ID", req.GetCollectionId()), + zap.Int64("partition ID", req.GetPartitionId()), + zap.String("channel name", req.GetChannelName()), + zap.Int64("# of rows", req.GetRowNum())) + errResp := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "", + } + if s.isClosed() { + log.Warn("failed to add segment for closed server") + errResp.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID()) + return errResp, nil + } + ok, nodeID := s.channelManager.getNodeIDByChannelName(req.GetChannelName()) + if !ok { + log.Error("no DataNode found for channel", zap.String("channel name", req.GetChannelName())) + errResp.Reason = fmt.Sprint("no DataNode found for channel ", req.GetChannelName()) + return errResp, nil + } + s.cluster.AddSegment(s.ctx, nodeID, req) + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} diff --git a/internal/datacoord/session_manager.go b/internal/datacoord/session_manager.go index 9e2b667064..74299b84bd 100644 --- a/internal/datacoord/session_manager.go +++ b/internal/datacoord/session_manager.go @@ -33,8 +33,9 @@ import ( const ( flushTimeout = 5 * time.Second // TODO: evaluate and update import timeout. - importTimeout = 3 * time.Hour - reCollectTimeout = 5 * time.Second + importTimeout = 3 * time.Hour + reCollectTimeout = 5 * time.Second + addSegmentTimeout = 30 * time.Second ) // SessionManager provides the grpc interfaces of cluster @@ -203,6 +204,29 @@ func (c *SessionManager) execReCollectSegmentStats(ctx context.Context, nodeID i } } +// AddSegment calls DataNode with ID == `nodeID` to put the segment into this node. +func (c *SessionManager) AddSegment(ctx context.Context, nodeID int64, req *datapb.AddSegmentRequest) { + go c.execAddSegment(ctx, nodeID, req) +} + +func (c *SessionManager) execAddSegment(ctx context.Context, nodeID int64, req *datapb.AddSegmentRequest) { + cli, err := c.getClient(ctx, nodeID) + if err != nil { + log.Warn("failed to get client for AddSegment", zap.Int64("DataNode ID", nodeID), zap.Error(err)) + return + } + ctx, cancel := context.WithTimeout(ctx, addSegmentTimeout) + defer cancel() + req.Base.SourceID = Params.DataCoordCfg.GetNodeID() + resp, err := cli.AddSegment(ctx, req) + if err := VerifyResponse(resp, err); err != nil { + log.Warn("failed to add segment", zap.Int64("DataNode ID", nodeID), zap.Error(err)) + return + } + + log.Info("success to add segment", zap.Int64("DataNode ID", nodeID), zap.Any("add segment req", req)) +} + func (c *SessionManager) getClient(ctx context.Context, nodeID int64) (types.DataNode, error) { c.sessions.RLock() session, ok := c.sessions.data[nodeID] diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index bc75b9e7ee..8744afe19c 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -102,7 +102,6 @@ var Params paramtable.ComponentParam type DataNode struct { ctx context.Context cancel context.CancelFunc - NodeID UniqueID Role string State atomic.Value // internalpb.StateCode_Initializing @@ -633,7 +632,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen // It returns a list of segments to be sent. func (node *DataNode) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) { log.Info("start resending segment stats, if any", - zap.Int64("DataNode ID", node.NodeID)) + zap.Int64("DataNode ID", Params.DataNodeCfg.GetNodeID())) segResent := node.flowgraphManager.resendTT() log.Info("found segment(s) with stats to resend", zap.Int64s("segment IDs", segResent)) @@ -811,7 +810,7 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) ErrorCode: commonpb.ErrorCode_Success, }, TaskId: req.GetImportTask().TaskId, - DatanodeId: node.NodeID, + DatanodeId: Params.DataNodeCfg.GetNodeID(), State: commonpb.ImportState_ImportStarted, Segments: make([]int64, 0), AutoIds: make([]int64, 0), @@ -843,7 +842,7 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) MsgType: commonpb.MsgType_RequestTSO, MsgID: 0, Timestamp: 0, - SourceID: node.NodeID, + SourceID: Params.DataNodeCfg.GetNodeID(), }, Count: 1, }) @@ -865,7 +864,7 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) ts := rep.GetTimestamp() metaService := newMetaService(node.rootCoord, req.GetImportTask().GetCollectionId()) - schema, err := metaService.getCollectionSchema(ctx, req.GetImportTask().GetCollectionId(), 0) + colInfo, err := metaService.getCollectionInfo(ctx, req.GetImportTask().GetCollectionId(), 0) if err != nil { importResult.State = commonpb.ImportState_ImportFailed importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: "failed_reason", Value: err.Error()}) @@ -882,8 +881,8 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) defer idAllocator.Close() segmentSize := int64(Params.DataCoordCfg.SegmentMaxSize) * 1024 * 1024 - importWrapper := importutil.NewImportWrapper(ctx, schema, 2, segmentSize, idAllocator, node.chunkManager, - importFlushReqFunc(node, req, importResult, schema, ts), importResult, reportFunc) + importWrapper := importutil.NewImportWrapper(ctx, colInfo.GetSchema(), colInfo.GetShardsNum(), segmentSize, idAllocator, node.chunkManager, + importFlushReqFunc(node, req, importResult, colInfo.GetSchema(), ts), importResult, reportFunc) err = importWrapper.Import(req.GetImportTask().GetFiles(), req.GetImportTask().GetRowBased(), false) if err != nil { importResult.State = commonpb.ImportState_ImportFailed @@ -901,6 +900,54 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) return resp, nil } +// AddSegment adds the segment to the current DataNode. +func (node *DataNode) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + log.Info("adding segment to DataNode flow graph", + zap.Int64("segment ID", req.GetSegmentId()), + zap.Int64("collection ID", req.GetCollectionId()), + zap.Int64("partition ID", req.GetPartitionId()), + zap.String("channel name", req.GetChannelName()), + zap.Int64("# of rows", req.GetRowNum())) + // Fetch the flow graph on the given v-channel. + ds, ok := node.flowgraphManager.getFlowgraphService(req.GetChannelName()) + if !ok { + log.Error("channel not found in current DataNode", + zap.String("channel name", req.GetChannelName()), + zap.Int64("node ID", Params.DataNodeCfg.GetNodeID())) + return &commonpb.Status{ + // TODO: Add specific error code. + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, nil + } + // Add the new segment to the replica. + if !ds.replica.hasSegment(req.GetSegmentId(), true) { + log.Info("add a new segment to replica") + err := ds.replica.addNewSegment(req.GetSegmentId(), + req.GetCollectionId(), + req.GetPartitionId(), + req.GetChannelName(), + &internalpb.MsgPosition{ + ChannelName: req.GetChannelName(), + }, + &internalpb.MsgPosition{ + ChannelName: req.GetChannelName(), + }) + if err != nil { + log.Error("failed to add segment to flow graph", + zap.Error(err)) + return &commonpb.Status{ + // TODO: Add specific error code. + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, nil + } + } + // Update # of rows of the given segment. + ds.replica.updateStatistics(req.GetSegmentId(), req.GetRowNum()) + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} + func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoordpb.ImportResult, schema *schemapb.CollectionSchema, ts Timestamp) importutil.ImportFlushFunc { return func(fields map[storage.FieldID]storage.FieldData, shardNum int) error { if shardNum >= len(req.GetImportTask().GetChannelNames()) { @@ -1053,39 +1100,20 @@ func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *root fieldStats = append(fieldStats, &datapb.FieldBinlog{FieldID: k, Binlogs: []*datapb.Binlog{v}}) } - ds, ok := node.flowgraphManager.getFlowgraphService(segReqs[0].GetChannelName()) - if !ok { - log.Warn("channel not found in current dataNode", - zap.String("channel name", segReqs[0].GetChannelName()), - zap.Int64("node ID", node.NodeID)) - return errors.New("channel " + segReqs[0].GetChannelName() + " not found in current dataNode") - } + log.Info("now adding segment to the correct DataNode flow graph") + // Ask DataCoord to add segment to the corresponding DataNode flow graph. + node.dataCoord.AddSegment(context.Background(), &datapb.AddSegmentRequest{ + Base: &commonpb.MsgBase{ + SourceID: Params.DataNodeCfg.GetNodeID(), + }, + SegmentId: segmentID, + ChannelName: segReqs[0].GetChannelName(), + CollectionId: req.GetImportTask().GetCollectionId(), + PartitionId: req.GetImportTask().GetPartitionId(), + RowNum: int64(rowNum), + }) - // Update flow graph replica segment info. - // TODO: Duplicate code. Add wrapper function. - if !ds.replica.hasSegment(segmentID, true) { - err = ds.replica.addNewSegment(segmentID, - req.GetImportTask().GetCollectionId(), - req.GetImportTask().GetPartitionId(), - segReqs[0].GetChannelName(), - &internalpb.MsgPosition{ - ChannelName: segReqs[0].GetChannelName(), - }, - &internalpb.MsgPosition{ - ChannelName: segReqs[0].GetChannelName(), - }) - if err != nil { - log.Error("failed to add segment", - zap.Int64("segment ID", segmentID), - zap.Int64("collection ID", req.GetImportTask().GetCollectionId()), - zap.Int64("partition ID", req.GetImportTask().GetPartitionId()), - zap.String("channel mame", segReqs[0].GetChannelName()), - zap.Error(err)) - } - } - ds.replica.updateStatistics(segmentID, int64(rowNum)) - - req := &datapb.SaveBinlogPathsRequest{ + binlogReq := &datapb.SaveBinlogPathsRequest{ Base: &commonpb.MsgBase{ MsgType: 0, //TODO msg type MsgID: 0, //TODO msg id @@ -1093,14 +1121,14 @@ func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *root SourceID: Params.DataNodeCfg.GetNodeID(), }, SegmentID: segmentID, - CollectionID: req.ImportTask.GetCollectionId(), + CollectionID: req.GetImportTask().GetCollectionId(), Field2BinlogPaths: fieldInsert, Field2StatslogPaths: fieldStats, Importing: true, } err = retry.Do(context.Background(), func() error { - rsp, err := node.dataCoord.SaveBinlogPaths(context.Background(), req) + rsp, err := node.dataCoord.SaveBinlogPaths(context.Background(), binlogReq) // should be network issue, return error and retry if err != nil { return fmt.Errorf(err.Error()) diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index e3457112f8..10dc023128 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -495,6 +495,75 @@ func TestDataNode(t *testing.T) { }) } +func TestDataNode_AddSegment(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + 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() + + node.chunkManager = storage.NewLocalChunkManager(storage.RootPath("/tmp/lib/milvus")) + Params.DataNodeCfg.SetNodeID(1) + + t.Run("test AddSegment", func(t *testing.T) { + node.rootCoord = &RootCoordFactory{ + collectionID: 100, + pkType: schemapb.DataType_Int64, + } + + chName1 := "fake-by-dev-rootcoord-dml-testaddsegment-1" + chName2 := "fake-by-dev-rootcoord-dml-testaddsegment-2" + err := node.flowgraphManager.addAndStart(node, &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: chName1, + UnflushedSegments: []*datapb.SegmentInfo{}, + FlushedSegments: []*datapb.SegmentInfo{}, + }) + require.Nil(t, err) + err = node.flowgraphManager.addAndStart(node, &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: chName2, + UnflushedSegments: []*datapb.SegmentInfo{}, + FlushedSegments: []*datapb.SegmentInfo{}, + }) + require.Nil(t, err) + + _, ok := node.flowgraphManager.getFlowgraphService(chName1) + assert.True(t, ok) + _, ok = node.flowgraphManager.getFlowgraphService(chName2) + assert.True(t, ok) + + stat, err := node.AddSegment(context.WithValue(ctx, ctxKey{}, ""), &datapb.AddSegmentRequest{ + SegmentId: 100, + CollectionId: 100, + PartitionId: 100, + ChannelName: chName1, + RowNum: 500, + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, stat.GetErrorCode()) + assert.Equal(t, "", stat.GetReason()) + + stat, err = node.AddSegment(context.WithValue(ctx, ctxKey{}, ""), &datapb.AddSegmentRequest{ + SegmentId: 100, + CollectionId: 100, + PartitionId: 100, + ChannelName: "bad-ch-name", + RowNum: 500, + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, stat.GetErrorCode()) + }) +} + func TestWatchChannel(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) diff --git a/internal/datanode/meta_service.go b/internal/datanode/meta_service.go index ccbe02b84c..649ebd4ce2 100644 --- a/internal/datanode/meta_service.go +++ b/internal/datanode/meta_service.go @@ -46,6 +46,7 @@ func newMetaService(rc types.RootCoord, collectionID UniqueID) *metaService { } } +// TODO: Replace with getCollectionInfo below. // getCollectionSchema get collection schema with provided collection id at specified timestamp. func (mService *metaService) getCollectionSchema(ctx context.Context, collID UniqueID, timestamp Timestamp) (*schemapb.CollectionSchema, error) { req := &milvuspb.DescribeCollectionRequest{ @@ -72,6 +73,32 @@ func (mService *metaService) getCollectionSchema(ctx context.Context, collID Uni return response.GetSchema(), nil } +// getCollectionInfo get collection info with provided collection id at specified timestamp. +func (mService *metaService) getCollectionInfo(ctx context.Context, collID UniqueID, timestamp Timestamp) (*milvuspb.DescribeCollectionResponse, error) { + req := &milvuspb.DescribeCollectionRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_DescribeCollection, + MsgID: 0, //GOOSE TODO + Timestamp: 0, // GOOSE TODO + SourceID: Params.DataNodeCfg.GetNodeID(), + }, + DbName: "default", // GOOSE TODO + CollectionID: collID, + TimeStamp: timestamp, + } + + response, err := mService.rootCoord.DescribeCollection(ctx, req) + if err != nil { + return nil, fmt.Errorf("grpc error when describe collection %v from rootcoord: %s", collID, err.Error()) + } + + if response.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + return nil, fmt.Errorf("describe collection %v from rootcoord wrong: %s", collID, response.GetStatus().GetReason()) + } + + return response, nil +} + // printCollectionStruct util function to print schema data, used in tests only. func printCollectionStruct(obj *etcdpb.CollectionMeta) { v := reflect.ValueOf(obj) diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index 9ab3be0e50..ab74c0cb4c 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -222,6 +222,12 @@ func (ds *DataCoordFactory) UpdateSegmentStatistics(ctx context.Context, req *da }, nil } +func (ds *DataCoordFactory) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} + func (mf *MetaFactory) GetCollectionMeta(collectionID UniqueID, collectionName string, pkDataType schemapb.DataType) *etcdpb.CollectionMeta { sch := schemapb.CollectionSchema{ Name: collectionName, @@ -918,6 +924,7 @@ func (m *RootCoordFactory) DescribeCollection(ctx context.Context, in *milvuspb. resp.CollectionID = m.collectionID resp.Schema = meta.Schema + resp.ShardsNum = 2 resp.Status.ErrorCode = commonpb.ErrorCode_Success return resp, nil } diff --git a/internal/datanode/segment_replica.go b/internal/datanode/segment_replica.go index 48daa3283c..a4cd1ac356 100644 --- a/internal/datanode/segment_replica.go +++ b/internal/datanode/segment_replica.go @@ -227,7 +227,7 @@ func (replica *SegmentReplica) normal2FlushedSegment(segID UniqueID) { replica.flushedSegments[segID] = &seg delete(replica.normalSegments, segID) - metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.NodeID)).Dec() + metrics.DataNodeNumUnflushedSegments.WithLabelValues(fmt.Sprint(Params.DataNodeCfg.GetNodeID())).Dec() } func (replica *SegmentReplica) getCollectionAndPartitionID(segID UniqueID) (collID, partitionID UniqueID, err error) { diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index 4a36c56b05..c1a66fc2fb 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -559,3 +559,17 @@ func (c *Client) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegm } return ret.(*commonpb.Status), err } + +// AddSegment is the DataCoord client side code for AddSegment call. +func (c *Client) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*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.DataCoordClient).AddSegment(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index 01a32a770c..6bca8e625d 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -135,6 +135,9 @@ func Test_NewClient(t *testing.T) { r26, err := client.ReleaseSegmentLock(ctx, nil) retCheck(retNotNil, r26, err) + + r27, err := client.AddSegment(ctx, nil) + retCheck(retNotNil, r27, err) } client.grpcClient = &mock.GRPCClientBase{ diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index 8cebd0b0cb..e19cc59bac 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -350,3 +350,7 @@ func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegm func (s *Server) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { return s.dataCoord.ReleaseSegmentLock(ctx, req) } + +func (s *Server) AddSegment(ctx context.Context, request *datapb.AddSegmentRequest) (*commonpb.Status, error) { + return s.dataCoord.AddSegment(ctx, request) +} diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index 21b847eaf8..f569043008 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -60,6 +60,7 @@ type MockDataCoord struct { updateSegStatResp *commonpb.Status acquireSegLockResp *commonpb.Status releaseSegLockResp *commonpb.Status + addSegmentResp *commonpb.Status } func (m *MockDataCoord) Init() error { @@ -189,6 +190,10 @@ func (m *MockDataCoord) ReleaseSegmentLock(ctx context.Context, req *datapb.Rele return m.releaseSegLockResp, m.err } +func (m *MockDataCoord) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + return m.addSegmentResp, m.err +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// func Test_NewServer(t *testing.T) { ctx := context.Background() @@ -452,6 +457,17 @@ func Test_NewServer(t *testing.T) { assert.NotNil(t, resp) }) + t.Run("add segment", func(t *testing.T) { + server.dataCoord = &MockDataCoord{ + addSegmentResp: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + } + resp, err := server.AddSegment(ctx, nil) + assert.Nil(t, err) + assert.NotNil(t, resp) + }) + err := server.Stop() assert.Nil(t, err) } diff --git a/internal/distributed/datanode/client/client.go b/internal/distributed/datanode/client/client.go index 32a02fc48e..48ae519a61 100644 --- a/internal/distributed/datanode/client/client.go +++ b/internal/distributed/datanode/client/client.go @@ -211,3 +211,17 @@ func (c *Client) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegme } return ret.(*datapb.ResendSegmentStatsResponse), err } + +// AddSegment is the DataNode client side code for AddSegment call. +func (c *Client) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*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).AddSegment(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} diff --git a/internal/distributed/datanode/client/client_test.go b/internal/distributed/datanode/client/client_test.go index 85257b211c..7a698e9500 100644 --- a/internal/distributed/datanode/client/client_test.go +++ b/internal/distributed/datanode/client/client_test.go @@ -82,6 +82,9 @@ func Test_NewClient(t *testing.T) { r8, err := client.ResendSegmentStats(ctx, nil) retCheck(retNotNil, r8, err) + + r9, err := client.AddSegment(ctx, nil) + retCheck(retNotNil, r9, err) } client.grpcClient = &mock.GRPCClientBase{ diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index c0a97bdfca..5691195a24 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -361,3 +361,7 @@ func (s *Server) Import(ctx context.Context, request *datapb.ImportTaskRequest) func (s *Server) ResendSegmentStats(ctx context.Context, request *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) { return s.datanode.ResendSegmentStats(ctx, request) } + +func (s *Server) AddSegment(ctx context.Context, request *datapb.AddSegmentRequest) (*commonpb.Status, error) { + return s.datanode.AddSegment(ctx, request) +} diff --git a/internal/distributed/datanode/service_test.go b/internal/distributed/datanode/service_test.go index 7d96bdc0f0..5778c74626 100644 --- a/internal/distributed/datanode/service_test.go +++ b/internal/distributed/datanode/service_test.go @@ -117,7 +117,11 @@ func (m *MockDataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest } func (m *MockDataNode) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) { - return m.resendResp, nil + return m.resendResp, m.err +} + +func (m *MockDataNode) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + return m.status, m.err } /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// @@ -273,6 +277,15 @@ func Test_NewServer(t *testing.T) { assert.NotNil(t, resp) }) + t.Run("add segment", func(t *testing.T) { + server.datanode = &MockDataNode{ + status: &commonpb.Status{}, + } + resp, err := server.AddSegment(ctx, nil) + assert.Nil(t, err) + assert.NotNil(t, resp) + }) + err = server.Stop() assert.Nil(t, err) } diff --git a/internal/distributed/proxy/service_test.go b/internal/distributed/proxy/service_test.go index dfbb044fc5..f9ae640dc1 100644 --- a/internal/distributed/proxy/service_test.go +++ b/internal/distributed/proxy/service_test.go @@ -438,6 +438,10 @@ func (m *MockDataCoord) Flush(ctx context.Context, req *datapb.FlushRequest) (*d return nil, nil } +func (m *MockDataCoord) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + return nil, nil +} + func (m *MockDataCoord) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) { return nil, nil } diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index b9076a174d..695a5141f5 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -52,6 +52,8 @@ service DataCoord { rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {} rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {} + + rpc AddSegment(AddSegmentRequest) returns(common.Status) {} } service DataNode { @@ -69,6 +71,8 @@ service DataNode { rpc Import(ImportTaskRequest) returns(common.Status) {} rpc ResendSegmentStats(ResendSegmentStatsRequest) returns(ResendSegmentStatsResponse) {} + + rpc AddSegment(AddSegmentRequest) returns(common.Status) {} } message FlushRequest { @@ -512,3 +516,12 @@ message ResendSegmentStatsResponse { common.Status status = 1; repeated int64 seg_resent = 2; } + +message AddSegmentRequest { + common.MsgBase base = 1; + int64 segment_id = 2; + string channel_name = 3; + int64 collection_id = 4; + int64 partition_id = 5; + int64 row_num = 6; +} diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index d9be729e8a..8cf6b8b031 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -3917,6 +3917,85 @@ func (m *ResendSegmentStatsResponse) GetSegResent() []int64 { return nil } +type AddSegmentRequest struct { + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + SegmentId int64 `protobuf:"varint,2,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` + ChannelName string `protobuf:"bytes,3,opt,name=channel_name,json=channelName,proto3" json:"channel_name,omitempty"` + CollectionId int64 `protobuf:"varint,4,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + PartitionId int64 `protobuf:"varint,5,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + RowNum int64 `protobuf:"varint,6,opt,name=row_num,json=rowNum,proto3" json:"row_num,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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{63} +} + +func (m *AddSegmentRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_AddSegmentRequest.Unmarshal(m, b) +} +func (m *AddSegmentRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_AddSegmentRequest.Marshal(b, m, deterministic) +} +func (m *AddSegmentRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AddSegmentRequest.Merge(m, src) +} +func (m *AddSegmentRequest) XXX_Size() int { + return xxx_messageInfo_AddSegmentRequest.Size(m) +} +func (m *AddSegmentRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AddSegmentRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AddSegmentRequest proto.InternalMessageInfo + +func (m *AddSegmentRequest) GetBase() *commonpb.MsgBase { + if m != nil { + return m.Base + } + return nil +} + +func (m *AddSegmentRequest) GetSegmentId() int64 { + if m != nil { + return m.SegmentId + } + return 0 +} + +func (m *AddSegmentRequest) GetChannelName() string { + if m != nil { + return m.ChannelName + } + return "" +} + +func (m *AddSegmentRequest) GetCollectionId() int64 { + if m != nil { + return m.CollectionId + } + return 0 +} + +func (m *AddSegmentRequest) GetPartitionId() int64 { + if m != nil { + return m.PartitionId + } + return 0 +} + +func (m *AddSegmentRequest) GetRowNum() int64 { + if m != nil { + return m.RowNum + } + return 0 +} + func init() { proto.RegisterEnum("milvus.proto.data.ChannelWatchState", ChannelWatchState_name, ChannelWatchState_value) proto.RegisterEnum("milvus.proto.data.CompactionType", CompactionType_name, CompactionType_value) @@ -3983,236 +4062,240 @@ func init() { proto.RegisterType((*UpdateSegmentStatisticsRequest)(nil), "milvus.proto.data.UpdateSegmentStatisticsRequest") proto.RegisterType((*ResendSegmentStatsRequest)(nil), "milvus.proto.data.ResendSegmentStatsRequest") proto.RegisterType((*ResendSegmentStatsResponse)(nil), "milvus.proto.data.ResendSegmentStatsResponse") + proto.RegisterType((*AddSegmentRequest)(nil), "milvus.proto.data.AddSegmentRequest") } func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 3579 bytes of a gzipped FileDescriptorProto + // 3628 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3b, 0x4b, 0x6f, 0x1b, 0xd7, - 0xd5, 0x1e, 0xbe, 0x44, 0x1e, 0x3e, 0x44, 0x5f, 0x3b, 0x32, 0x4d, 0xdb, 0xb2, 0x3c, 0x8e, 0x1d, - 0xc5, 0x71, 0xec, 0x44, 0x4e, 0x10, 0xe3, 0xcb, 0x0b, 0xb1, 0x65, 0x2b, 0xc4, 0x27, 0xb9, 0xca, - 0x48, 0x89, 0x8b, 0xa6, 0xe8, 0x60, 0xc4, 0xb9, 0xa2, 0x26, 0xe2, 0xcc, 0xd0, 0x33, 0x43, 0xcb, - 0xca, 0x26, 0x46, 0x83, 0x16, 0x4d, 0x51, 0xf4, 0x81, 0x6e, 0xda, 0xa2, 0x8b, 0xa2, 0xab, 0x3e, - 0x36, 0x05, 0x82, 0x2e, 0xda, 0xa2, 0xfb, 0xa0, 0x5d, 0xf4, 0x27, 0x74, 0xd9, 0x5d, 0x7f, 0x43, - 0x71, 0x1f, 0x73, 0xe7, 0xc1, 0x19, 0x72, 0x44, 0xda, 0xf1, 0x4e, 0xf7, 0xcc, 0x39, 0xe7, 0x9e, - 0x7b, 0xee, 0x79, 0xf3, 0x0a, 0x9a, 0xba, 0xe6, 0x69, 0x6a, 0xd7, 0xb6, 0x1d, 0xfd, 0xda, 0xc0, - 0xb1, 0x3d, 0x1b, 0x1d, 0x37, 0x8d, 0xfe, 0xc3, 0xa1, 0xcb, 0x56, 0xd7, 0xc8, 0xe7, 0x76, 0xad, - 0x6b, 0x9b, 0xa6, 0x6d, 0x31, 0x50, 0xbb, 0x61, 0x58, 0x1e, 0x76, 0x2c, 0xad, 0xcf, 0xd7, 0xb5, - 0x30, 0x41, 0xbb, 0xe6, 0x76, 0xf7, 0xb0, 0xa9, 0xb1, 0x95, 0x3c, 0x07, 0xc5, 0x3b, 0xe6, 0xc0, - 0x3b, 0x94, 0x7f, 0x21, 0x41, 0xed, 0x6e, 0x7f, 0xe8, 0xee, 0x29, 0xf8, 0xc1, 0x10, 0xbb, 0x1e, - 0x7a, 0x05, 0x0a, 0x3b, 0x9a, 0x8b, 0x5b, 0xd2, 0x92, 0xb4, 0x5c, 0x5d, 0x39, 0x7b, 0x2d, 0xb2, - 0x2b, 0xdf, 0x6f, 0xc3, 0xed, 0xdd, 0xd2, 0x5c, 0xac, 0x50, 0x4c, 0x84, 0xa0, 0xa0, 0xef, 0x74, - 0x56, 0x5b, 0xb9, 0x25, 0x69, 0x39, 0xaf, 0xd0, 0xbf, 0xd1, 0x22, 0x80, 0x8b, 0x7b, 0x26, 0xb6, - 0xbc, 0xce, 0xaa, 0xdb, 0xca, 0x2f, 0xe5, 0x97, 0xf3, 0x4a, 0x08, 0x82, 0x64, 0xa8, 0x75, 0xed, - 0x7e, 0x1f, 0x77, 0x3d, 0xc3, 0xb6, 0x3a, 0xab, 0xad, 0x02, 0xa5, 0x8d, 0xc0, 0xe4, 0x5f, 0x4b, - 0x50, 0xe7, 0xa2, 0xb9, 0x03, 0xdb, 0x72, 0x31, 0xba, 0x01, 0x25, 0xd7, 0xd3, 0xbc, 0xa1, 0xcb, - 0xa5, 0x3b, 0x93, 0x28, 0xdd, 0x16, 0x45, 0x51, 0x38, 0x6a, 0xa2, 0x78, 0xf1, 0xed, 0xf3, 0xa3, - 0xdb, 0xc7, 0x8e, 0x50, 0x88, 0x1f, 0x41, 0xfe, 0x99, 0x04, 0xcd, 0x2d, 0x7f, 0xe9, 0x6b, 0xef, - 0x24, 0x14, 0xbb, 0xf6, 0xd0, 0xf2, 0xa8, 0x80, 0x75, 0x85, 0x2d, 0xd0, 0x05, 0xa8, 0x75, 0xf7, - 0x34, 0xcb, 0xc2, 0x7d, 0xd5, 0xd2, 0x4c, 0x4c, 0x45, 0xa9, 0x28, 0x55, 0x0e, 0xbb, 0xa7, 0x99, - 0x38, 0x93, 0x44, 0x4b, 0x50, 0x1d, 0x68, 0x8e, 0x67, 0x44, 0x74, 0x16, 0x06, 0xc9, 0xbf, 0x91, - 0x60, 0xe1, 0x3d, 0xd7, 0x35, 0x7a, 0xd6, 0x88, 0x64, 0x0b, 0x50, 0xb2, 0x6c, 0x1d, 0x77, 0x56, - 0xa9, 0x68, 0x79, 0x85, 0xaf, 0xd0, 0x19, 0xa8, 0x0c, 0x30, 0x76, 0x54, 0xc7, 0xee, 0xfb, 0x82, - 0x95, 0x09, 0x40, 0xb1, 0xfb, 0x18, 0x7d, 0x00, 0xc7, 0xdd, 0x18, 0x23, 0x76, 0x9b, 0xd5, 0x95, - 0x8b, 0xd7, 0x46, 0xec, 0xf1, 0x5a, 0x7c, 0x53, 0x65, 0x94, 0x5a, 0x7e, 0x9c, 0x83, 0x13, 0x02, - 0x8f, 0xc9, 0x4a, 0xfe, 0x26, 0x9a, 0x73, 0x71, 0x4f, 0x88, 0xc7, 0x16, 0x59, 0x34, 0x27, 0x54, - 0x9e, 0x0f, 0xab, 0x3c, 0x83, 0x81, 0xc5, 0xf5, 0x59, 0x1c, 0xd1, 0x27, 0x3a, 0x0f, 0x55, 0xfc, - 0x68, 0x60, 0x38, 0x58, 0xf5, 0x0c, 0x13, 0xb7, 0x4a, 0x4b, 0xd2, 0x72, 0x41, 0x01, 0x06, 0xda, - 0x36, 0xcc, 0xb0, 0x45, 0xce, 0x65, 0xb6, 0x48, 0xf9, 0xb7, 0x12, 0x9c, 0x1a, 0xb9, 0x25, 0x6e, - 0xe2, 0x0a, 0x34, 0xe9, 0xc9, 0x03, 0xcd, 0x10, 0x63, 0x27, 0x0a, 0xbf, 0x3c, 0x4e, 0xe1, 0x01, - 0xba, 0x32, 0x42, 0x1f, 0x12, 0x32, 0x97, 0x5d, 0xc8, 0x7d, 0x38, 0xb5, 0x86, 0x3d, 0xbe, 0x01, - 0xf9, 0x86, 0xdd, 0xe9, 0x43, 0x44, 0xd4, 0x97, 0x72, 0x23, 0xbe, 0xf4, 0xa7, 0x9c, 0xf0, 0x25, - 0xba, 0x55, 0xc7, 0xda, 0xb5, 0xd1, 0x59, 0xa8, 0x08, 0x14, 0x6e, 0x15, 0x01, 0x00, 0xbd, 0x01, - 0x45, 0x22, 0x29, 0x33, 0x89, 0xc6, 0xca, 0x85, 0xe4, 0x33, 0x85, 0x78, 0x2a, 0x0c, 0x1f, 0x75, - 0xa0, 0xe1, 0x7a, 0x9a, 0xe3, 0xa9, 0x03, 0xdb, 0xa5, 0xf7, 0x4c, 0x0d, 0xa7, 0xba, 0x22, 0x47, - 0x39, 0x88, 0x60, 0xba, 0xe1, 0xf6, 0x36, 0x39, 0xa6, 0x52, 0xa7, 0x94, 0xfe, 0x12, 0xdd, 0x81, - 0x1a, 0xb6, 0xf4, 0x80, 0x51, 0x21, 0x33, 0xa3, 0x2a, 0xb6, 0x74, 0xc1, 0x26, 0xb8, 0x9f, 0x62, - 0xf6, 0xfb, 0xf9, 0x91, 0x04, 0xad, 0xd1, 0x0b, 0x9a, 0x25, 0x50, 0xbe, 0xc9, 0x88, 0x30, 0xbb, - 0xa0, 0xb1, 0x1e, 0x2e, 0x2e, 0x49, 0xe1, 0x24, 0xb2, 0x01, 0xcf, 0x05, 0xd2, 0xd0, 0x2f, 0x4f, - 0xcd, 0x58, 0x3e, 0x97, 0x60, 0x21, 0xbe, 0xd7, 0x2c, 0xe7, 0x7e, 0x0d, 0x8a, 0x86, 0xb5, 0x6b, - 0xfb, 0xc7, 0x5e, 0x1c, 0xe3, 0x67, 0x64, 0x2f, 0x86, 0x2c, 0x9b, 0x70, 0x66, 0x0d, 0x7b, 0x1d, - 0xcb, 0xc5, 0x8e, 0x77, 0xcb, 0xb0, 0xfa, 0x76, 0x6f, 0x53, 0xf3, 0xf6, 0x66, 0xf0, 0x91, 0x88, - 0xb9, 0xe7, 0x62, 0xe6, 0x2e, 0xff, 0x4e, 0x82, 0xb3, 0xc9, 0xfb, 0xf1, 0xa3, 0xb7, 0xa1, 0xbc, - 0x6b, 0xe0, 0xbe, 0x4e, 0x74, 0x26, 0x51, 0x9d, 0x89, 0x35, 0xf1, 0x95, 0x01, 0x41, 0xe6, 0x27, - 0xbc, 0x90, 0x62, 0xa0, 0x5b, 0x9e, 0x63, 0x58, 0xbd, 0x75, 0xc3, 0xf5, 0x14, 0x86, 0x1f, 0xd2, - 0x67, 0x3e, 0xbb, 0x65, 0xfe, 0x50, 0x82, 0xc5, 0x35, 0xec, 0xdd, 0x16, 0xa1, 0x96, 0x7c, 0x37, - 0x5c, 0xcf, 0xe8, 0xba, 0x4f, 0xb6, 0xc8, 0xc8, 0x90, 0x33, 0xe5, 0x9f, 0x48, 0x70, 0x3e, 0x55, - 0x18, 0xae, 0x3a, 0x1e, 0x4a, 0xfc, 0x40, 0x9b, 0x1c, 0x4a, 0xfe, 0x1f, 0x1f, 0x7e, 0xa4, 0xf5, - 0x87, 0x78, 0x53, 0x33, 0x1c, 0x16, 0x4a, 0xa6, 0x0c, 0xac, 0x7f, 0x94, 0xe0, 0xdc, 0x1a, 0xf6, - 0x36, 0xfd, 0x34, 0xf3, 0x0c, 0xb5, 0x93, 0xa1, 0xa2, 0xf8, 0x31, 0xbb, 0xcc, 0x44, 0x69, 0x9f, - 0x89, 0xfa, 0x16, 0xa9, 0x1f, 0x84, 0x1c, 0xf2, 0x36, 0xab, 0x05, 0xb8, 0xf2, 0xe4, 0xef, 0x49, - 0x70, 0xfa, 0xbd, 0xee, 0x83, 0xa1, 0xe1, 0x60, 0x8e, 0xb4, 0x6e, 0x77, 0xf7, 0xa7, 0x57, 0x6d, - 0x50, 0x37, 0xe5, 0x22, 0x75, 0xd3, 0x84, 0x0a, 0x97, 0xca, 0xa1, 0xe0, 0x3e, 0xd6, 0xdc, 0x67, - 0x2b, 0xc7, 0xe3, 0x3c, 0xd4, 0x3e, 0xe2, 0xf5, 0x12, 0x4d, 0xab, 0x71, 0xbb, 0x90, 0x92, 0xed, - 0x22, 0x54, 0x62, 0x25, 0x55, 0x5d, 0x6b, 0x50, 0x77, 0x31, 0xde, 0x9f, 0x26, 0x89, 0xd6, 0x08, - 0xa1, 0x48, 0x7e, 0xeb, 0x70, 0x7c, 0x68, 0xed, 0x92, 0x32, 0x1f, 0xeb, 0x5c, 0x51, 0xac, 0xda, - 0x9e, 0x1c, 0x89, 0x47, 0x09, 0xd1, 0xfb, 0x30, 0x1f, 0xe7, 0x55, 0xcc, 0xc4, 0x2b, 0x4e, 0x86, - 0x3a, 0xd0, 0xd4, 0x1d, 0x7b, 0x30, 0xc0, 0xba, 0xea, 0xfa, 0xac, 0x4a, 0xd9, 0x58, 0x71, 0x3a, - 0x9f, 0x95, 0xfc, 0x85, 0x04, 0x0b, 0xf7, 0x35, 0xaf, 0xbb, 0xb7, 0x6a, 0x72, 0x63, 0x9d, 0xc1, - 0xd5, 0xdf, 0x86, 0xca, 0x43, 0x7e, 0x11, 0x7e, 0x3c, 0x3f, 0x9f, 0x20, 0x50, 0xf8, 0xca, 0x95, - 0x80, 0x42, 0xfe, 0x4a, 0x82, 0x93, 0xb4, 0xa9, 0xf2, 0xa5, 0xfb, 0xfa, 0x83, 0xce, 0x84, 0xc6, - 0x0a, 0x5d, 0x86, 0x86, 0xa9, 0x39, 0xfb, 0x5b, 0x01, 0x4e, 0x91, 0xe2, 0xc4, 0xa0, 0xf2, 0x23, - 0x00, 0xbe, 0xda, 0x70, 0x7b, 0x53, 0xc8, 0x7f, 0x13, 0xe6, 0xf8, 0xae, 0x3c, 0xfe, 0x4c, 0xba, - 0x58, 0x1f, 0x5d, 0xfe, 0x87, 0x04, 0x8d, 0x20, 0xa3, 0x50, 0xaf, 0x6a, 0x40, 0x4e, 0xf8, 0x52, - 0xae, 0xb3, 0x8a, 0xde, 0x86, 0x12, 0x6b, 0xb8, 0x39, 0xef, 0x4b, 0x51, 0xde, 0xbc, 0x19, 0x0f, - 0xa5, 0x25, 0x0a, 0x50, 0x38, 0x11, 0xd1, 0x91, 0x88, 0xc2, 0xc2, 0xab, 0x03, 0x08, 0xea, 0xc0, - 0x7c, 0xb4, 0x88, 0xf5, 0x7d, 0x66, 0x29, 0x2d, 0xfa, 0xae, 0x6a, 0x9e, 0x46, 0x83, 0x6f, 0x23, - 0x52, 0xc3, 0xba, 0xf2, 0x7f, 0x8b, 0x50, 0x0d, 0x9d, 0x72, 0xe4, 0x24, 0xf1, 0x2b, 0xcd, 0x4d, - 0xce, 0x23, 0xf9, 0xd1, 0x4e, 0xea, 0x12, 0x34, 0x0c, 0x5a, 0xbb, 0xa8, 0xdc, 0x14, 0x69, 0xb2, - 0xa9, 0x28, 0x75, 0x06, 0xe5, 0x7e, 0x81, 0x16, 0xa1, 0x6a, 0x0d, 0x4d, 0xd5, 0xde, 0x55, 0x1d, - 0xfb, 0xc0, 0xe5, 0x2d, 0x59, 0xc5, 0x1a, 0x9a, 0xdf, 0xd8, 0x55, 0xec, 0x03, 0x37, 0xa8, 0xfa, - 0x4b, 0x47, 0xac, 0xfa, 0x17, 0xa1, 0x6a, 0x6a, 0x8f, 0x08, 0x57, 0xd5, 0x1a, 0x9a, 0xb4, 0x5b, - 0xcb, 0x2b, 0x15, 0x53, 0x7b, 0xa4, 0xd8, 0x07, 0xf7, 0x86, 0x26, 0x5a, 0x86, 0x66, 0x5f, 0x73, - 0x3d, 0x35, 0xdc, 0xee, 0x95, 0x69, 0xbb, 0xd7, 0x20, 0xf0, 0x3b, 0x41, 0xcb, 0x37, 0xda, 0x3f, - 0x54, 0x66, 0xe8, 0x1f, 0x74, 0xb3, 0x1f, 0x30, 0x82, 0xec, 0xfd, 0x83, 0x6e, 0xf6, 0x05, 0x9b, - 0x9b, 0x30, 0xb7, 0x43, 0x2b, 0x42, 0xb7, 0x55, 0x4d, 0x8d, 0x50, 0x77, 0x49, 0x31, 0xc8, 0x0a, - 0x47, 0xc5, 0x47, 0x47, 0x6f, 0x41, 0x85, 0xa6, 0x62, 0x4a, 0x5b, 0xcb, 0x44, 0x1b, 0x10, 0x10, - 0x6a, 0x1d, 0xf7, 0x3d, 0x8d, 0x52, 0xd7, 0xb3, 0x51, 0x0b, 0x02, 0xf4, 0x0a, 0x9c, 0xe8, 0x3a, - 0x58, 0xf3, 0xb0, 0x7e, 0xeb, 0xf0, 0xb6, 0x6d, 0x0e, 0x34, 0x6a, 0x4c, 0xad, 0xc6, 0x92, 0xb4, - 0x5c, 0x56, 0x92, 0x3e, 0x91, 0xc0, 0xd0, 0x15, 0xab, 0xbb, 0x8e, 0x6d, 0xb6, 0xe6, 0x59, 0x60, - 0x88, 0x42, 0xd1, 0x39, 0x00, 0x3f, 0x74, 0x6b, 0x5e, 0xab, 0x49, 0x6f, 0xb1, 0xc2, 0x21, 0xef, - 0x79, 0xf2, 0x67, 0x70, 0x32, 0xb0, 0x90, 0xd0, 0x6d, 0x8c, 0x5e, 0xac, 0x34, 0xed, 0xc5, 0x8e, - 0xaf, 0xe5, 0xff, 0x55, 0x80, 0x85, 0x2d, 0xed, 0x21, 0x7e, 0xfa, 0x6d, 0x43, 0xa6, 0x78, 0xbc, - 0x0e, 0xc7, 0x69, 0xa7, 0xb0, 0x12, 0x92, 0x67, 0x4c, 0x06, 0x0e, 0x5f, 0xe7, 0x28, 0x21, 0x7a, - 0x97, 0x94, 0x0e, 0xb8, 0xbb, 0xbf, 0x69, 0x1b, 0x41, 0xf6, 0x3d, 0x97, 0xc0, 0xe7, 0xb6, 0xc0, - 0x52, 0xc2, 0x14, 0x68, 0x73, 0x34, 0xb4, 0xb1, 0xbc, 0xfb, 0xc2, 0xd8, 0x7e, 0x34, 0xd0, 0x7e, - 0x3c, 0xc2, 0xa1, 0x16, 0xcc, 0xf1, 0xec, 0x4e, 0xfd, 0xbe, 0xac, 0xf8, 0x4b, 0xb4, 0x09, 0x27, - 0xd8, 0x09, 0xb6, 0xb8, 0x51, 0xb3, 0xc3, 0x97, 0x33, 0x1d, 0x3e, 0x89, 0x34, 0xea, 0x13, 0x95, - 0xa3, 0xfa, 0x44, 0x0b, 0xe6, 0xb8, 0x9d, 0xd2, 0x58, 0x50, 0x56, 0xfc, 0x25, 0xb9, 0x66, 0xc3, - 0x1c, 0xd8, 0x8e, 0x67, 0x58, 0xbd, 0x56, 0x95, 0x7e, 0x0b, 0x00, 0xa4, 0xe5, 0x82, 0x40, 0x9f, - 0x13, 0x26, 0x27, 0xef, 0x40, 0x59, 0x58, 0x78, 0x2e, 0xb3, 0x85, 0x0b, 0x9a, 0x78, 0x8c, 0xce, - 0xc7, 0x62, 0xb4, 0xfc, 0x4f, 0x09, 0x6a, 0xab, 0xe4, 0x48, 0xeb, 0x76, 0x8f, 0x66, 0x94, 0x4b, - 0xd0, 0x70, 0x70, 0xd7, 0x76, 0x74, 0x15, 0x5b, 0x9e, 0x63, 0x60, 0xd6, 0x9d, 0x17, 0x94, 0x3a, - 0x83, 0xde, 0x61, 0x40, 0x82, 0x46, 0xc2, 0xae, 0xeb, 0x69, 0xe6, 0x40, 0xdd, 0x25, 0xee, 0x9d, - 0x63, 0x68, 0x02, 0x4a, 0xbd, 0xfb, 0x02, 0xd4, 0x02, 0x34, 0xcf, 0xa6, 0xfb, 0x17, 0x94, 0xaa, - 0x80, 0x6d, 0xdb, 0xe8, 0x79, 0x68, 0x50, 0x9d, 0xaa, 0x7d, 0xbb, 0xa7, 0x92, 0x4e, 0x96, 0x27, - 0x9b, 0x9a, 0xce, 0xc5, 0x22, 0x77, 0x15, 0xc5, 0x72, 0x8d, 0x4f, 0x31, 0x4f, 0x37, 0x02, 0x6b, - 0xcb, 0xf8, 0x14, 0x93, 0x5c, 0x5f, 0x27, 0xb9, 0xf3, 0x9e, 0xad, 0xe3, 0xed, 0x29, 0x2b, 0x8d, - 0x0c, 0x53, 0xcc, 0xb3, 0x50, 0x11, 0x27, 0xe0, 0x47, 0x0a, 0x00, 0xe8, 0x2e, 0x34, 0xfc, 0x22, - 0x54, 0x65, 0xbd, 0x56, 0x21, 0xb5, 0xf2, 0x0b, 0x65, 0x3f, 0x57, 0xa9, 0xfb, 0x64, 0x74, 0x29, - 0xdf, 0x85, 0x5a, 0xf8, 0x33, 0xd9, 0x75, 0x2b, 0x6e, 0x28, 0x02, 0x40, 0xac, 0xf1, 0xde, 0xd0, - 0x24, 0x77, 0xca, 0x03, 0x8b, 0xbf, 0x94, 0x3f, 0x97, 0xa0, 0xce, 0x53, 0xf6, 0x96, 0x98, 0xb2, - 0xd3, 0xa3, 0x49, 0xf4, 0x68, 0xf4, 0x6f, 0xf4, 0x7f, 0xd1, 0x11, 0xdd, 0xf3, 0x89, 0x41, 0x80, - 0x32, 0xa1, 0xd5, 0x71, 0x24, 0x5f, 0x67, 0xe9, 0xed, 0x1f, 0x13, 0x43, 0xe3, 0x57, 0x43, 0x0d, - 0xad, 0x05, 0x73, 0x9a, 0xae, 0x3b, 0xd8, 0x75, 0xb9, 0x1c, 0xfe, 0x92, 0x7c, 0x79, 0x88, 0x1d, - 0xd7, 0x37, 0xf9, 0xbc, 0xe2, 0x2f, 0xd1, 0x5b, 0x50, 0x16, 0xe5, 0x74, 0x3e, 0xa9, 0x84, 0x0a, - 0xcb, 0xc9, 0x7b, 0x51, 0x41, 0x21, 0xff, 0x39, 0x07, 0x0d, 0xae, 0xb0, 0x5b, 0x3c, 0xa7, 0x8e, - 0x77, 0xbe, 0x5b, 0x50, 0xdb, 0x0d, 0x7c, 0x7f, 0xdc, 0xcc, 0x29, 0x1c, 0x22, 0x22, 0x34, 0x93, - 0x1c, 0x30, 0x9a, 0xd5, 0x0b, 0x33, 0x65, 0xf5, 0xe2, 0x51, 0x23, 0xd8, 0x68, 0x9d, 0x57, 0x4a, - 0xa8, 0xf3, 0xe4, 0x6f, 0x43, 0x35, 0xc4, 0x80, 0x46, 0x68, 0x36, 0xac, 0xe2, 0x1a, 0xf3, 0x97, - 0xe8, 0x46, 0x50, 0xdb, 0x30, 0x55, 0x9d, 0x4e, 0x90, 0x25, 0x56, 0xd6, 0xc8, 0xbf, 0x97, 0xa0, - 0xc4, 0x39, 0x9f, 0x87, 0x2a, 0x0f, 0x3a, 0xb4, 0xee, 0x63, 0xdc, 0x81, 0x83, 0x48, 0xe1, 0xf7, - 0xe4, 0xa2, 0xce, 0x69, 0x28, 0xc7, 0xe2, 0xcd, 0x1c, 0x4f, 0x0b, 0xfe, 0xa7, 0x50, 0x90, 0x21, - 0x9f, 0x68, 0x7c, 0xf9, 0x4a, 0xa2, 0x83, 0x76, 0x05, 0x77, 0xed, 0x87, 0xd8, 0x39, 0x9c, 0x7d, - 0x9c, 0xf9, 0x66, 0xc8, 0xa0, 0x33, 0xf6, 0x87, 0x82, 0x00, 0xbd, 0x19, 0xa8, 0x3b, 0x9f, 0x34, - 0xcd, 0x09, 0x47, 0x18, 0x6e, 0x8e, 0x81, 0xda, 0x7f, 0xca, 0x06, 0xb3, 0xd1, 0xa3, 0x4c, 0x5b, - 0xd7, 0x3c, 0x91, 0xb6, 0x43, 0xfe, 0xb9, 0x04, 0xa7, 0xd7, 0xb0, 0x77, 0x37, 0xda, 0xdc, 0x3f, - 0x6b, 0xa9, 0x4c, 0x68, 0x27, 0x09, 0x35, 0xcb, 0xad, 0xb7, 0xa1, 0x2c, 0xc6, 0x14, 0x6c, 0x64, - 0x2e, 0xd6, 0xf2, 0xf7, 0x25, 0x68, 0xf1, 0x5d, 0xe8, 0x9e, 0xa4, 0xa4, 0xee, 0x63, 0x0f, 0xeb, - 0x5f, 0x77, 0xdf, 0xfc, 0x77, 0x09, 0x9a, 0xe1, 0x88, 0x4f, 0x83, 0xf6, 0xeb, 0x50, 0xa4, 0xe3, - 0x09, 0x2e, 0xc1, 0x44, 0x63, 0x65, 0xd8, 0x24, 0x64, 0xd0, 0x32, 0x6f, 0x5b, 0x24, 0x27, 0xbe, - 0x0c, 0xd2, 0x4e, 0xfe, 0xe8, 0x69, 0x87, 0xa7, 0x61, 0x7b, 0x48, 0xf8, 0xb2, 0x71, 0x68, 0x00, - 0x90, 0xbf, 0xcc, 0x41, 0x2b, 0xe8, 0x47, 0xbe, 0xf6, 0xb8, 0x9f, 0x52, 0xad, 0xe6, 0x9f, 0x50, - 0xb5, 0x5a, 0x98, 0x3d, 0xd6, 0x17, 0x93, 0x62, 0xfd, 0xdf, 0x72, 0xd0, 0x08, 0xb4, 0xb6, 0xd9, - 0xd7, 0x2c, 0xb4, 0x00, 0xa5, 0x41, 0x5f, 0x0b, 0xa6, 0x8f, 0x7c, 0x85, 0xb6, 0x44, 0x9d, 0x13, - 0xd5, 0xd3, 0x4b, 0x49, 0x77, 0x98, 0x72, 0x11, 0x4a, 0x8c, 0x05, 0x69, 0x07, 0x59, 0x43, 0x41, - 0x9b, 0x7a, 0x5e, 0x5b, 0x31, 0x63, 0x21, 0xfd, 0xfc, 0x55, 0x40, 0xfc, 0x86, 0x55, 0xc3, 0x52, - 0x5d, 0xdc, 0xb5, 0x2d, 0x9d, 0xdd, 0x7d, 0x51, 0x69, 0xf2, 0x2f, 0x1d, 0x6b, 0x8b, 0xc1, 0xd1, - 0xeb, 0x50, 0xf0, 0x0e, 0x07, 0x2c, 0x8a, 0x37, 0x12, 0xa3, 0x63, 0x20, 0xd7, 0xf6, 0xe1, 0x00, - 0x2b, 0x14, 0x1d, 0x2d, 0x02, 0x10, 0x56, 0x9e, 0xa3, 0x3d, 0xe4, 0x29, 0xb1, 0xa0, 0x84, 0x20, - 0xc4, 0x9a, 0x7d, 0x1d, 0xce, 0xb1, 0xd4, 0xc1, 0x97, 0xf2, 0x5f, 0x72, 0xd0, 0x0c, 0x58, 0x2a, - 0xd8, 0x1d, 0xf6, 0xbd, 0x54, 0xfd, 0x8d, 0x6f, 0x06, 0x27, 0xd5, 0x0d, 0xef, 0x42, 0x95, 0xdf, - 0xe7, 0x11, 0xec, 0x01, 0x18, 0xc9, 0xfa, 0x18, 0x03, 0x2d, 0x3e, 0x21, 0x03, 0x2d, 0x1d, 0xd1, - 0x40, 0xe5, 0x2d, 0x58, 0xf0, 0xe3, 0x5e, 0x80, 0xb0, 0x81, 0x3d, 0x6d, 0x4c, 0xc1, 0x71, 0x1e, - 0xaa, 0x2c, 0x9f, 0xb1, 0x44, 0xce, 0x4a, 0x75, 0xd8, 0x11, 0x1d, 0xae, 0xfc, 0x1d, 0x38, 0x49, - 0xe3, 0x46, 0x7c, 0x94, 0x9b, 0x65, 0xae, 0x2e, 0x8b, 0x46, 0x80, 0x14, 0xfd, 0xcc, 0xba, 0x2b, - 0x4a, 0x04, 0x26, 0xaf, 0xc3, 0x73, 0x31, 0xfe, 0x33, 0xe4, 0x05, 0x52, 0x0a, 0x2d, 0x6c, 0x45, - 0x7f, 0x26, 0x9e, 0x3e, 0xfb, 0x9d, 0x13, 0x93, 0x5b, 0xd5, 0xd0, 0xe3, 0xf6, 0xa5, 0xa3, 0x77, - 0xa0, 0x62, 0xe1, 0x03, 0x35, 0x1c, 0x7c, 0x33, 0x0c, 0xe8, 0xca, 0x16, 0x3e, 0xa0, 0x7f, 0xc9, - 0xf7, 0xe0, 0xd4, 0x88, 0xa8, 0xb3, 0x9c, 0xfd, 0xaf, 0x12, 0x9c, 0x5e, 0x75, 0xec, 0xc1, 0x47, - 0x86, 0xe3, 0x0d, 0xb5, 0x7e, 0xf4, 0x87, 0xa2, 0xa7, 0xd3, 0xc6, 0xbd, 0x1f, 0x4a, 0xc3, 0x2c, - 0x2e, 0x5f, 0x4d, 0x30, 0xd7, 0x51, 0xa1, 0xf8, 0xa1, 0x43, 0x49, 0xfb, 0x3f, 0xf9, 0x24, 0xe1, - 0x39, 0xde, 0x84, 0x64, 0x93, 0xa5, 0x4a, 0x49, 0x9c, 0xfa, 0xe4, 0xa7, 0x9d, 0xfa, 0xa4, 0x78, - 0x7e, 0xe1, 0x09, 0x79, 0xfe, 0x91, 0xdb, 0x90, 0xf7, 0x21, 0x3a, 0x91, 0xa3, 0x21, 0x77, 0xaa, - 0x51, 0xde, 0x2d, 0x80, 0x60, 0x3a, 0xc5, 0x5f, 0xf9, 0x64, 0x61, 0x13, 0xa2, 0x22, 0xb7, 0x25, - 0xa2, 0x2c, 0x9d, 0x2a, 0x47, 0xe6, 0x25, 0x1f, 0x40, 0x3b, 0xc9, 0x4a, 0x67, 0xb1, 0xfc, 0x2f, - 0x73, 0x00, 0x1d, 0x3a, 0x1d, 0xda, 0xd6, 0xdc, 0xfd, 0xe9, 0x2a, 0xca, 0x8b, 0x50, 0x0f, 0x0c, - 0x26, 0xf0, 0xf7, 0xb0, 0x15, 0xe9, 0xc4, 0x25, 0x44, 0x61, 0x4b, 0x70, 0x46, 0x8a, 0x5d, 0x9d, - 0xf2, 0x09, 0x79, 0x0d, 0x33, 0x8a, 0x58, 0xd0, 0x43, 0x67, 0xa0, 0xe2, 0xd8, 0x07, 0x2a, 0x71, - 0x33, 0x9d, 0xe6, 0xd6, 0xb2, 0x52, 0x76, 0xec, 0x03, 0xe2, 0x7c, 0x3a, 0x3a, 0x05, 0x73, 0x9e, - 0xe6, 0xee, 0x13, 0xfe, 0x25, 0x96, 0xee, 0xc8, 0xb2, 0xa3, 0xa3, 0x93, 0x50, 0xdc, 0x35, 0xfa, - 0xd8, 0x6d, 0xcd, 0x51, 0x96, 0x6c, 0x81, 0xde, 0xf0, 0xdf, 0x73, 0x94, 0x33, 0xff, 0x1e, 0xcd, - 0x9e, 0x74, 0x7c, 0x25, 0xc1, 0x7c, 0xa0, 0x35, 0x1a, 0x80, 0x48, 0x4c, 0xa3, 0xf1, 0xec, 0xb6, - 0xad, 0xb3, 0x50, 0xd1, 0x48, 0xf9, 0x89, 0x85, 0x11, 0xb2, 0xa8, 0x15, 0x90, 0x8c, 0xab, 0xcb, - 0xc9, 0xb9, 0xc8, 0xa1, 0x0d, 0xdd, 0xff, 0x85, 0xa7, 0xe4, 0xd8, 0x07, 0x1d, 0x5d, 0x68, 0x83, - 0x3d, 0x6b, 0x63, 0x55, 0x28, 0xd1, 0xc6, 0x6d, 0xfa, 0xb2, 0xed, 0x22, 0xd4, 0xb1, 0xe3, 0xd8, - 0x8e, 0x6a, 0x62, 0xd7, 0xd5, 0x7a, 0x98, 0x17, 0x5d, 0x35, 0x0a, 0xdc, 0x60, 0x30, 0xf9, 0xdf, - 0x79, 0x68, 0x04, 0x47, 0xf1, 0x7f, 0xd7, 0x31, 0x74, 0xff, 0x77, 0x1d, 0x43, 0x27, 0xc1, 0xdc, - 0x61, 0xa1, 0x30, 0x14, 0xcc, 0x39, 0xa4, 0xa3, 0x93, 0x3c, 0x48, 0x1c, 0xcc, 0xb2, 0x75, 0x1c, - 0x5c, 0x2c, 0xf8, 0x20, 0x7e, 0xaf, 0x11, 0xfb, 0x28, 0x64, 0xb0, 0x8f, 0x62, 0x06, 0xfb, 0x28, - 0x25, 0xd8, 0xc7, 0x02, 0x94, 0x76, 0x86, 0xdd, 0x7d, 0xec, 0xf1, 0xf2, 0x88, 0xaf, 0xa2, 0x76, - 0x53, 0x8e, 0xd9, 0x8d, 0x30, 0x8f, 0x4a, 0xd8, 0x3c, 0xce, 0x40, 0x85, 0xfd, 0xb8, 0xa0, 0x7a, - 0x2e, 0x9d, 0xb2, 0xe6, 0x95, 0x32, 0x03, 0x6c, 0xbb, 0xe8, 0xa6, 0xdf, 0x3b, 0x54, 0x93, 0x1c, - 0x9d, 0x46, 0x9c, 0x98, 0x85, 0xf8, 0x9d, 0xc3, 0x4d, 0x68, 0xed, 0xe1, 0xa1, 0x43, 0xdf, 0x46, - 0xa8, 0xf4, 0xf1, 0xee, 0x83, 0x21, 0x76, 0x0e, 0xb5, 0x9d, 0x3e, 0x6e, 0xd5, 0xa8, 0x60, 0x0b, - 0xe2, 0xfb, 0xaa, 0xe6, 0x69, 0x1f, 0xf8, 0x5f, 0xd1, 0x6b, 0xb0, 0x10, 0xa3, 0x34, 0x2c, 0x1d, - 0x3f, 0xc2, 0x7a, 0xab, 0x4e, 0xe9, 0x4e, 0x46, 0xe8, 0x3a, 0xec, 0x9b, 0xfc, 0x09, 0xa0, 0x40, - 0x92, 0xd9, 0x7a, 0xc7, 0xd8, 0x55, 0xe7, 0xe2, 0x57, 0x2d, 0xff, 0x41, 0x82, 0xe3, 0xe1, 0xcd, - 0xa6, 0x4d, 0xa0, 0xef, 0x40, 0x95, 0xcd, 0xac, 0x55, 0xe2, 0xc0, 0xbc, 0x7b, 0x3c, 0x37, 0x56, - 0xc7, 0x0a, 0x18, 0x41, 0x1c, 0xbb, 0x08, 0xf5, 0x03, 0xdb, 0xd9, 0x37, 0xac, 0x9e, 0x4a, 0x24, - 0xf3, 0xdd, 0xa6, 0xc6, 0x81, 0xf7, 0x08, 0x4c, 0xfe, 0x42, 0x82, 0xc5, 0x0f, 0x07, 0xba, 0xe6, - 0xe1, 0x50, 0x25, 0x31, 0xeb, 0x03, 0x9b, 0xd7, 0xfd, 0x37, 0x2e, 0xb9, 0x6c, 0x73, 0x57, 0x86, - 0x2d, 0x6f, 0xc0, 0x69, 0x05, 0xbb, 0xd8, 0xd2, 0x23, 0x1f, 0xa7, 0x95, 0x42, 0x1e, 0x40, 0x3b, - 0x89, 0xdd, 0x2c, 0x77, 0xcf, 0x4a, 0x3a, 0xd5, 0x21, 0x6c, 0x3d, 0x1e, 0xa1, 0x48, 0x25, 0x41, - 0xf7, 0xf1, 0xae, 0xfc, 0x52, 0x82, 0xe3, 0x23, 0xcd, 0x32, 0x6a, 0x00, 0x7c, 0x68, 0x75, 0xf9, - 0x14, 0xa1, 0x79, 0x0c, 0xd5, 0xa0, 0xec, 0xcf, 0x14, 0x9a, 0x12, 0xaa, 0xc2, 0xdc, 0xb6, 0x4d, - 0xb1, 0x9b, 0x39, 0xd4, 0x84, 0x1a, 0x23, 0x1c, 0x76, 0xbb, 0xd8, 0x75, 0x9b, 0x79, 0x01, 0xb9, - 0xab, 0x19, 0xfd, 0xa1, 0x83, 0x9b, 0x05, 0x54, 0x87, 0xca, 0xb6, 0xcd, 0xdf, 0xca, 0x34, 0x8b, - 0x08, 0x41, 0xc3, 0x7f, 0x38, 0xc3, 0x89, 0x4a, 0x21, 0x98, 0x4f, 0x36, 0x77, 0x65, 0x37, 0xdc, - 0x56, 0x92, 0x5e, 0x0b, 0x9d, 0x82, 0x13, 0x1f, 0x5a, 0x3a, 0xde, 0x35, 0x2c, 0xac, 0x07, 0x9f, - 0x9a, 0xc7, 0xd0, 0x09, 0x98, 0xef, 0x58, 0x16, 0x76, 0x42, 0x40, 0x89, 0x00, 0x37, 0xb0, 0xd3, - 0xc3, 0x21, 0x60, 0x0e, 0x1d, 0x87, 0xfa, 0x86, 0xf1, 0x28, 0x04, 0xca, 0xaf, 0xfc, 0xe0, 0x14, - 0x54, 0x88, 0xe7, 0xdd, 0xb6, 0x6d, 0x47, 0x47, 0x03, 0x40, 0xf4, 0x3d, 0x99, 0x39, 0xb0, 0x2d, - 0xf1, 0xf0, 0x12, 0xbd, 0x92, 0x52, 0x11, 0x8c, 0xa2, 0xf2, 0xdb, 0x6f, 0x5f, 0x4e, 0xa1, 0x88, - 0xa1, 0xcb, 0xc7, 0x90, 0x49, 0x77, 0x24, 0xbd, 0xea, 0xb6, 0xd1, 0xdd, 0xf7, 0x7f, 0x29, 0x1f, - 0xb3, 0x63, 0x0c, 0xd5, 0xdf, 0x31, 0xf6, 0x9e, 0x93, 0x2f, 0xd8, 0xa3, 0x3f, 0xdf, 0x88, 0xe4, - 0x63, 0xe8, 0x01, 0x9c, 0x5c, 0xc3, 0x21, 0xa7, 0xf1, 0x37, 0x5c, 0x49, 0xdf, 0x70, 0x04, 0xf9, - 0x88, 0x5b, 0xae, 0x43, 0x91, 0x0e, 0xa6, 0x50, 0x92, 0x5f, 0x85, 0xff, 0x3b, 0xa1, 0xbd, 0x94, - 0x8e, 0x20, 0xb8, 0x7d, 0x02, 0xf3, 0xb1, 0xd7, 0xd5, 0xe8, 0xc5, 0x04, 0xb2, 0xe4, 0x77, 0xf2, - 0xed, 0x2b, 0x59, 0x50, 0xc5, 0x5e, 0x3d, 0x68, 0x44, 0x5f, 0xa3, 0xa1, 0xe5, 0x04, 0xfa, 0xc4, - 0x97, 0xb1, 0xed, 0x17, 0x33, 0x60, 0x8a, 0x8d, 0x4c, 0x68, 0xc6, 0x5f, 0xfb, 0xa2, 0x2b, 0x63, - 0x19, 0x44, 0xcd, 0xed, 0xa5, 0x4c, 0xb8, 0x62, 0xbb, 0x43, 0x6a, 0x04, 0x23, 0xaf, 0x4d, 0xd1, - 0xb5, 0x64, 0x36, 0x69, 0xcf, 0x60, 0xdb, 0xd7, 0x33, 0xe3, 0x8b, 0xad, 0xbf, 0xcb, 0x06, 0xe2, - 0x49, 0x2f, 0x36, 0xd1, 0xab, 0xc9, 0xec, 0xc6, 0x3c, 0x35, 0x6d, 0xaf, 0x1c, 0x85, 0x44, 0x08, - 0xf1, 0x19, 0x9d, 0x64, 0x27, 0xbc, 0x7a, 0x8c, 0xfb, 0x9d, 0xcf, 0x2f, 0xfd, 0x39, 0x67, 0xfb, - 0xd5, 0x23, 0x50, 0x08, 0x01, 0xec, 0xf8, 0x7b, 0x6a, 0xdf, 0x0d, 0xaf, 0x4f, 0xb4, 0x9a, 0xe9, - 0x7c, 0xf0, 0x63, 0x98, 0x8f, 0xbd, 0x49, 0x48, 0xf4, 0x9a, 0xe4, 0x77, 0x0b, 0xed, 0x71, 0xb9, - 0x86, 0xb9, 0x64, 0xec, 0x87, 0x01, 0x94, 0x62, 0xfd, 0x09, 0x3f, 0x1e, 0xb4, 0xaf, 0x64, 0x41, - 0x15, 0x07, 0x71, 0x69, 0xb8, 0x8c, 0x0d, 0xd7, 0xd1, 0xd5, 0x64, 0x1e, 0xc9, 0x3f, 0x0c, 0xb4, - 0x5f, 0xce, 0x88, 0x2d, 0x36, 0x55, 0x01, 0xd6, 0xb0, 0xb7, 0x81, 0x3d, 0x87, 0xd8, 0xc8, 0xe5, - 0x44, 0x95, 0x07, 0x08, 0xfe, 0x36, 0x2f, 0x4c, 0xc4, 0x13, 0x1b, 0x7c, 0x13, 0x90, 0x9f, 0x62, - 0x43, 0x2f, 0x62, 0x2e, 0x8e, 0x9d, 0x3f, 0xb2, 0x61, 0xe1, 0xa4, 0xbb, 0x79, 0x00, 0xcd, 0x0d, - 0xcd, 0x22, 0x9d, 0x67, 0xc0, 0xf7, 0x6a, 0xa2, 0x60, 0x71, 0xb4, 0x14, 0x6d, 0xa5, 0x62, 0x8b, - 0xc3, 0x1c, 0x88, 0x1c, 0xaa, 0x09, 0x17, 0xc4, 0xf1, 0xd8, 0x12, 0x68, 0x23, 0x86, 0x98, 0x12, - 0x5b, 0xc6, 0xe0, 0x8b, 0x8d, 0x1f, 0x4b, 0xf4, 0xd5, 0x7e, 0x0c, 0xe1, 0xbe, 0xe1, 0xed, 0x6d, - 0xf6, 0x35, 0xcb, 0xcd, 0x22, 0x02, 0x45, 0x3c, 0x82, 0x08, 0x1c, 0x5f, 0x88, 0xa0, 0x43, 0x3d, - 0x32, 0xde, 0x43, 0x49, 0xcf, 0x5a, 0x92, 0x06, 0x8c, 0xed, 0xe5, 0xc9, 0x88, 0x62, 0x97, 0x3d, - 0xa8, 0xfb, 0xf6, 0xca, 0x94, 0xfb, 0x62, 0x9a, 0xa4, 0x01, 0x4e, 0x8a, 0xbb, 0x25, 0xa3, 0x86, - 0xdd, 0x6d, 0x74, 0x7a, 0x81, 0xb2, 0x4d, 0xbd, 0xc6, 0xb9, 0x5b, 0xfa, 0x48, 0x84, 0xc5, 0x93, - 0xd8, 0xa4, 0x30, 0x39, 0x58, 0x25, 0x0e, 0x3e, 0x13, 0xe3, 0x49, 0xca, 0xe0, 0x51, 0x3e, 0x86, - 0xee, 0x43, 0x89, 0xb5, 0x23, 0xe8, 0xf9, 0xf1, 0x9d, 0x0a, 0xe7, 0x7e, 0x69, 0x02, 0x96, 0x60, - 0xbc, 0x0f, 0xa7, 0x52, 0xfa, 0x94, 0xc4, 0x3c, 0x37, 0xbe, 0xa7, 0x99, 0xe4, 0xe5, 0x1a, 0xa0, - 0xd1, 0x57, 0xf1, 0x89, 0xd7, 0x94, 0xfa, 0x78, 0x3e, 0xc3, 0x16, 0xa3, 0x0f, 0xde, 0x13, 0xb7, - 0x48, 0x7d, 0x17, 0x3f, 0x61, 0x8b, 0x95, 0x5f, 0x95, 0xa0, 0xec, 0xbf, 0xf8, 0x78, 0x06, 0x95, - 0xf8, 0x33, 0x28, 0x8d, 0x3f, 0x86, 0xf9, 0xd8, 0xd3, 0xf1, 0x44, 0x4b, 0x4f, 0x7e, 0x5e, 0x3e, - 0xe9, 0xc6, 0xee, 0xf3, 0x7f, 0xb0, 0x15, 0x59, 0xf2, 0x85, 0xb4, 0xf2, 0x3a, 0x9e, 0x20, 0x27, - 0x30, 0x7e, 0xea, 0xe9, 0xf0, 0x1e, 0x40, 0x28, 0x5d, 0x8d, 0xff, 0x19, 0x8e, 0x44, 0xe0, 0x49, - 0x02, 0x6f, 0x1c, 0xd1, 0xc9, 0x27, 0xb0, 0x73, 0x89, 0x2b, 0xc4, 0x1b, 0xf5, 0x14, 0x57, 0x48, - 0x19, 0x0f, 0x24, 0x06, 0xc5, 0xf4, 0xee, 0x5f, 0x3e, 0x76, 0xeb, 0xc6, 0xb7, 0x5e, 0xed, 0x19, - 0xde, 0xde, 0x70, 0x87, 0x88, 0x73, 0x9d, 0x11, 0xbf, 0x6c, 0xd8, 0xfc, 0xaf, 0xeb, 0xbe, 0x55, - 0x5e, 0xa7, 0xfc, 0xae, 0x13, 0x7e, 0x83, 0x9d, 0x9d, 0x12, 0x5d, 0xdd, 0xf8, 0x5f, 0x00, 0x00, - 0x00, 0xff, 0xff, 0x6e, 0xe3, 0xdf, 0x9f, 0x70, 0x3e, 0x00, 0x00, + 0xd5, 0x1e, 0xbe, 0x44, 0x1e, 0x3e, 0x44, 0x5d, 0x3b, 0x32, 0x4d, 0xdb, 0xb2, 0x3c, 0x8e, 0x1d, + 0xc5, 0x71, 0xec, 0x44, 0x4e, 0x10, 0xe3, 0xcb, 0x0b, 0xb6, 0x65, 0x2b, 0xc4, 0x27, 0xf9, 0x93, + 0x47, 0x4a, 0xfc, 0xa1, 0x29, 0x4a, 0x8c, 0x38, 0x57, 0xd4, 0x44, 0x9c, 0x19, 0x7a, 0x66, 0x68, + 0x59, 0xd9, 0xc4, 0x68, 0xd0, 0x02, 0x29, 0x8a, 0x3e, 0xd0, 0x4d, 0x0b, 0x74, 0x51, 0x74, 0xd5, + 0xc7, 0xa6, 0x40, 0xd0, 0x45, 0x5b, 0x74, 0x1f, 0xb4, 0x8b, 0xfe, 0x80, 0x2e, 0xba, 0x6c, 0x37, + 0xed, 0x6f, 0x28, 0xee, 0x63, 0xee, 0xbc, 0xc9, 0x91, 0x68, 0xc7, 0x3b, 0xde, 0x7b, 0xcf, 0x39, + 0xf7, 0xdc, 0x73, 0xcf, 0x7b, 0x2e, 0xa1, 0xa9, 0xa9, 0xae, 0xda, 0xed, 0x59, 0x96, 0xad, 0x5d, + 0x1d, 0xda, 0x96, 0x6b, 0xa1, 0x39, 0x43, 0x1f, 0x3c, 0x1a, 0x39, 0x6c, 0x74, 0x95, 0x2c, 0xb7, + 0x6b, 0x3d, 0xcb, 0x30, 0x2c, 0x93, 0x4d, 0xb5, 0x1b, 0xba, 0xe9, 0x62, 0xdb, 0x54, 0x07, 0x7c, + 0x5c, 0x0b, 0x22, 0xb4, 0x6b, 0x4e, 0x6f, 0x17, 0x1b, 0x2a, 0x1b, 0xc9, 0x33, 0x50, 0xbc, 0x63, + 0x0c, 0xdd, 0x03, 0xf9, 0xa7, 0x12, 0xd4, 0xee, 0x0e, 0x46, 0xce, 0xae, 0x82, 0x1f, 0x8e, 0xb0, + 0xe3, 0xa2, 0xd7, 0xa0, 0xb0, 0xad, 0x3a, 0xb8, 0x25, 0x2d, 0x4a, 0x4b, 0xd5, 0xe5, 0x33, 0x57, + 0x43, 0xbb, 0xf2, 0xfd, 0xd6, 0x9d, 0xfe, 0x2d, 0xd5, 0xc1, 0x0a, 0x85, 0x44, 0x08, 0x0a, 0xda, + 0x76, 0x67, 0xa5, 0x95, 0x5b, 0x94, 0x96, 0xf2, 0x0a, 0xfd, 0x8d, 0x16, 0x00, 0x1c, 0xdc, 0x37, + 0xb0, 0xe9, 0x76, 0x56, 0x9c, 0x56, 0x7e, 0x31, 0xbf, 0x94, 0x57, 0x02, 0x33, 0x48, 0x86, 0x5a, + 0xcf, 0x1a, 0x0c, 0x70, 0xcf, 0xd5, 0x2d, 0xb3, 0xb3, 0xd2, 0x2a, 0x50, 0xdc, 0xd0, 0x9c, 0xfc, + 0x73, 0x09, 0xea, 0x9c, 0x35, 0x67, 0x68, 0x99, 0x0e, 0x46, 0xd7, 0xa1, 0xe4, 0xb8, 0xaa, 0x3b, + 0x72, 0x38, 0x77, 0xa7, 0x13, 0xb9, 0xdb, 0xa4, 0x20, 0x0a, 0x07, 0x4d, 0x64, 0x2f, 0xba, 0x7d, + 0x3e, 0xbe, 0x7d, 0xe4, 0x08, 0x85, 0xe8, 0x11, 0xe4, 0x1f, 0x4b, 0xd0, 0xdc, 0xf4, 0x86, 0x9e, + 0xf4, 0x4e, 0x40, 0xb1, 0x67, 0x8d, 0x4c, 0x97, 0x32, 0x58, 0x57, 0xd8, 0x00, 0x9d, 0x87, 0x5a, + 0x6f, 0x57, 0x35, 0x4d, 0x3c, 0xe8, 0x9a, 0xaa, 0x81, 0x29, 0x2b, 0x15, 0xa5, 0xca, 0xe7, 0xee, + 0xa9, 0x06, 0xce, 0xc4, 0xd1, 0x22, 0x54, 0x87, 0xaa, 0xed, 0xea, 0x21, 0x99, 0x05, 0xa7, 0xe4, + 0x5f, 0x48, 0x30, 0x7f, 0xd3, 0x71, 0xf4, 0xbe, 0x19, 0xe3, 0x6c, 0x1e, 0x4a, 0xa6, 0xa5, 0xe1, + 0xce, 0x0a, 0x65, 0x2d, 0xaf, 0xf0, 0x11, 0x3a, 0x0d, 0x95, 0x21, 0xc6, 0x76, 0xd7, 0xb6, 0x06, + 0x1e, 0x63, 0x65, 0x32, 0xa1, 0x58, 0x03, 0x8c, 0xee, 0xc3, 0x9c, 0x13, 0x21, 0xc4, 0x6e, 0xb3, + 0xba, 0x7c, 0xe1, 0x6a, 0x4c, 0x1f, 0xaf, 0x46, 0x37, 0x55, 0xe2, 0xd8, 0xf2, 0x93, 0x1c, 0x1c, + 0x17, 0x70, 0x8c, 0x57, 0xf2, 0x9b, 0x48, 0xce, 0xc1, 0x7d, 0xc1, 0x1e, 0x1b, 0x64, 0x91, 0x9c, + 0x10, 0x79, 0x3e, 0x28, 0xf2, 0x0c, 0x0a, 0x16, 0x95, 0x67, 0x31, 0x26, 0x4f, 0x74, 0x0e, 0xaa, + 0xf8, 0xf1, 0x50, 0xb7, 0x71, 0xd7, 0xd5, 0x0d, 0xdc, 0x2a, 0x2d, 0x4a, 0x4b, 0x05, 0x05, 0xd8, + 0xd4, 0x96, 0x6e, 0x04, 0x35, 0x72, 0x26, 0xb3, 0x46, 0xca, 0xbf, 0x94, 0xe0, 0x64, 0xec, 0x96, + 0xb8, 0x8a, 0x2b, 0xd0, 0xa4, 0x27, 0xf7, 0x25, 0x43, 0x94, 0x9d, 0x08, 0xfc, 0xd2, 0x38, 0x81, + 0xfb, 0xe0, 0x4a, 0x0c, 0x3f, 0xc0, 0x64, 0x2e, 0x3b, 0x93, 0x7b, 0x70, 0x72, 0x15, 0xbb, 0x7c, + 0x03, 0xb2, 0x86, 0x9d, 0xa3, 0xbb, 0x88, 0xb0, 0x2d, 0xe5, 0x62, 0xb6, 0xf4, 0xbb, 0x9c, 0xb0, + 0x25, 0xba, 0x55, 0xc7, 0xdc, 0xb1, 0xd0, 0x19, 0xa8, 0x08, 0x10, 0xae, 0x15, 0xfe, 0x04, 0x7a, + 0x0b, 0x8a, 0x84, 0x53, 0xa6, 0x12, 0x8d, 0xe5, 0xf3, 0xc9, 0x67, 0x0a, 0xd0, 0x54, 0x18, 0x3c, + 0xea, 0x40, 0xc3, 0x71, 0x55, 0xdb, 0xed, 0x0e, 0x2d, 0x87, 0xde, 0x33, 0x55, 0x9c, 0xea, 0xb2, + 0x1c, 0xa6, 0x20, 0x9c, 0xe9, 0xba, 0xd3, 0xdf, 0xe0, 0x90, 0x4a, 0x9d, 0x62, 0x7a, 0x43, 0x74, + 0x07, 0x6a, 0xd8, 0xd4, 0x7c, 0x42, 0x85, 0xcc, 0x84, 0xaa, 0xd8, 0xd4, 0x04, 0x19, 0xff, 0x7e, + 0x8a, 0xd9, 0xef, 0xe7, 0xfb, 0x12, 0xb4, 0xe2, 0x17, 0x34, 0x8d, 0xa3, 0x7c, 0x9b, 0x21, 0x61, + 0x76, 0x41, 0x63, 0x2d, 0x5c, 0x5c, 0x92, 0xc2, 0x51, 0x64, 0x1d, 0x5e, 0xf0, 0xb9, 0xa1, 0x2b, + 0xcf, 0x4c, 0x59, 0x3e, 0x97, 0x60, 0x3e, 0xba, 0xd7, 0x34, 0xe7, 0x7e, 0x03, 0x8a, 0xba, 0xb9, + 0x63, 0x79, 0xc7, 0x5e, 0x18, 0x63, 0x67, 0x64, 0x2f, 0x06, 0x2c, 0x1b, 0x70, 0x7a, 0x15, 0xbb, + 0x1d, 0xd3, 0xc1, 0xb6, 0x7b, 0x4b, 0x37, 0x07, 0x56, 0x7f, 0x43, 0x75, 0x77, 0xa7, 0xb0, 0x91, + 0x90, 0xba, 0xe7, 0x22, 0xea, 0x2e, 0xff, 0x4a, 0x82, 0x33, 0xc9, 0xfb, 0xf1, 0xa3, 0xb7, 0xa1, + 0xbc, 0xa3, 0xe3, 0x81, 0x46, 0x64, 0x26, 0x51, 0x99, 0x89, 0x31, 0xb1, 0x95, 0x21, 0x01, 0xe6, + 0x27, 0x3c, 0x9f, 0xa2, 0xa0, 0x9b, 0xae, 0xad, 0x9b, 0xfd, 0x35, 0xdd, 0x71, 0x15, 0x06, 0x1f, + 0x90, 0x67, 0x3e, 0xbb, 0x66, 0x7e, 0x4f, 0x82, 0x85, 0x55, 0xec, 0xde, 0x16, 0xae, 0x96, 0xac, + 0xeb, 0x8e, 0xab, 0xf7, 0x9c, 0xa7, 0x9b, 0x64, 0x64, 0x88, 0x99, 0xf2, 0x0f, 0x25, 0x38, 0x97, + 0xca, 0x0c, 0x17, 0x1d, 0x77, 0x25, 0x9e, 0xa3, 0x4d, 0x76, 0x25, 0xff, 0x8b, 0x0f, 0x3e, 0x52, + 0x07, 0x23, 0xbc, 0xa1, 0xea, 0x36, 0x73, 0x25, 0x47, 0x74, 0xac, 0xbf, 0x95, 0xe0, 0xec, 0x2a, + 0x76, 0x37, 0xbc, 0x30, 0xf3, 0x1c, 0xa5, 0x93, 0x21, 0xa3, 0xf8, 0x01, 0xbb, 0xcc, 0x44, 0x6e, + 0x9f, 0x8b, 0xf8, 0x16, 0xa8, 0x1d, 0x04, 0x0c, 0xf2, 0x36, 0xcb, 0x05, 0xb8, 0xf0, 0xe4, 0xef, + 0x48, 0x70, 0xea, 0x66, 0xef, 0xe1, 0x48, 0xb7, 0x31, 0x07, 0x5a, 0xb3, 0x7a, 0x7b, 0x47, 0x17, + 0xad, 0x9f, 0x37, 0xe5, 0x42, 0x79, 0xd3, 0x84, 0x0c, 0x97, 0xf2, 0xa1, 0xe0, 0x01, 0x56, 0x9d, + 0xe7, 0xcb, 0xc7, 0x93, 0x3c, 0xd4, 0x3e, 0xe2, 0xf9, 0x12, 0x0d, 0xab, 0x51, 0xbd, 0x90, 0x92, + 0xf5, 0x22, 0x90, 0x62, 0x25, 0x65, 0x5d, 0xab, 0x50, 0x77, 0x30, 0xde, 0x3b, 0x4a, 0x10, 0xad, + 0x11, 0x44, 0x11, 0xfc, 0xd6, 0x60, 0x6e, 0x64, 0xee, 0x90, 0x34, 0x1f, 0x6b, 0x5c, 0x50, 0x2c, + 0xdb, 0x9e, 0xec, 0x89, 0xe3, 0x88, 0xe8, 0x03, 0x98, 0x8d, 0xd2, 0x2a, 0x66, 0xa2, 0x15, 0x45, + 0x43, 0x1d, 0x68, 0x6a, 0xb6, 0x35, 0x1c, 0x62, 0xad, 0xeb, 0x78, 0xa4, 0x4a, 0xd9, 0x48, 0x71, + 0x3c, 0x8f, 0x94, 0xfc, 0x85, 0x04, 0xf3, 0x0f, 0x54, 0xb7, 0xb7, 0xbb, 0x62, 0x70, 0x65, 0x9d, + 0xc2, 0xd4, 0xdf, 0x85, 0xca, 0x23, 0x7e, 0x11, 0x9e, 0x3f, 0x3f, 0x97, 0xc0, 0x50, 0xf0, 0xca, + 0x15, 0x1f, 0x43, 0xfe, 0x4a, 0x82, 0x13, 0xb4, 0xa8, 0xf2, 0xb8, 0xfb, 0xfa, 0x9d, 0xce, 0x84, + 0xc2, 0x0a, 0x5d, 0x82, 0x86, 0xa1, 0xda, 0x7b, 0x9b, 0x3e, 0x4c, 0x91, 0xc2, 0x44, 0x66, 0xe5, + 0xc7, 0x00, 0x7c, 0xb4, 0xee, 0xf4, 0x8f, 0xc0, 0xff, 0x0d, 0x98, 0xe1, 0xbb, 0x72, 0xff, 0x33, + 0xe9, 0x62, 0x3d, 0x70, 0xf9, 0x2f, 0x12, 0x34, 0xfc, 0x88, 0x42, 0xad, 0xaa, 0x01, 0x39, 0x61, + 0x4b, 0xb9, 0xce, 0x0a, 0x7a, 0x17, 0x4a, 0xac, 0xe0, 0xe6, 0xb4, 0x2f, 0x86, 0x69, 0xf3, 0x62, + 0x3c, 0x10, 0x96, 0xe8, 0x84, 0xc2, 0x91, 0x88, 0x8c, 0x84, 0x17, 0x16, 0x56, 0xed, 0xcf, 0xa0, + 0x0e, 0xcc, 0x86, 0x93, 0x58, 0xcf, 0x66, 0x16, 0xd3, 0xbc, 0xef, 0x8a, 0xea, 0xaa, 0xd4, 0xf9, + 0x36, 0x42, 0x39, 0xac, 0x23, 0xff, 0xa7, 0x08, 0xd5, 0xc0, 0x29, 0x63, 0x27, 0x89, 0x5e, 0x69, + 0x6e, 0x72, 0x1c, 0xc9, 0xc7, 0x2b, 0xa9, 0x8b, 0xd0, 0xd0, 0x69, 0xee, 0xd2, 0xe5, 0xaa, 0x48, + 0x83, 0x4d, 0x45, 0xa9, 0xb3, 0x59, 0x6e, 0x17, 0x68, 0x01, 0xaa, 0xe6, 0xc8, 0xe8, 0x5a, 0x3b, + 0x5d, 0xdb, 0xda, 0x77, 0x78, 0x49, 0x56, 0x31, 0x47, 0xc6, 0xff, 0xed, 0x28, 0xd6, 0xbe, 0xe3, + 0x67, 0xfd, 0xa5, 0x43, 0x66, 0xfd, 0x0b, 0x50, 0x35, 0xd4, 0xc7, 0x84, 0x6a, 0xd7, 0x1c, 0x19, + 0xb4, 0x5a, 0xcb, 0x2b, 0x15, 0x43, 0x7d, 0xac, 0x58, 0xfb, 0xf7, 0x46, 0x06, 0x5a, 0x82, 0xe6, + 0x40, 0x75, 0xdc, 0x6e, 0xb0, 0xdc, 0x2b, 0xd3, 0x72, 0xaf, 0x41, 0xe6, 0xef, 0xf8, 0x25, 0x5f, + 0xbc, 0x7e, 0xa8, 0x4c, 0x51, 0x3f, 0x68, 0xc6, 0xc0, 0x27, 0x04, 0xd9, 0xeb, 0x07, 0xcd, 0x18, + 0x08, 0x32, 0x37, 0x60, 0x66, 0x9b, 0x66, 0x84, 0x4e, 0xab, 0x9a, 0xea, 0xa1, 0xee, 0x92, 0x64, + 0x90, 0x25, 0x8e, 0x8a, 0x07, 0x8e, 0xde, 0x81, 0x0a, 0x0d, 0xc5, 0x14, 0xb7, 0x96, 0x09, 0xd7, + 0x47, 0x20, 0xd8, 0x1a, 0x1e, 0xb8, 0x2a, 0xc5, 0xae, 0x67, 0xc3, 0x16, 0x08, 0xe8, 0x35, 0x38, + 0xde, 0xb3, 0xb1, 0xea, 0x62, 0xed, 0xd6, 0xc1, 0x6d, 0xcb, 0x18, 0xaa, 0x54, 0x99, 0x5a, 0x8d, + 0x45, 0x69, 0xa9, 0xac, 0x24, 0x2d, 0x11, 0xc7, 0xd0, 0x13, 0xa3, 0xbb, 0xb6, 0x65, 0xb4, 0x66, + 0x99, 0x63, 0x08, 0xcf, 0xa2, 0xb3, 0x00, 0x9e, 0xeb, 0x56, 0xdd, 0x56, 0x93, 0xde, 0x62, 0x85, + 0xcf, 0xdc, 0x74, 0xe5, 0xcf, 0xe0, 0x84, 0xaf, 0x21, 0x81, 0xdb, 0x88, 0x5f, 0xac, 0x74, 0xd4, + 0x8b, 0x1d, 0x9f, 0xcb, 0xff, 0xad, 0x00, 0xf3, 0x9b, 0xea, 0x23, 0xfc, 0xec, 0xcb, 0x86, 0x4c, + 0xfe, 0x78, 0x0d, 0xe6, 0x68, 0xa5, 0xb0, 0x1c, 0xe0, 0x67, 0x4c, 0x04, 0x0e, 0x5e, 0x67, 0x1c, + 0x11, 0xbd, 0x4f, 0x52, 0x07, 0xdc, 0xdb, 0xdb, 0xb0, 0x74, 0x3f, 0xfa, 0x9e, 0x4d, 0xa0, 0x73, + 0x5b, 0x40, 0x29, 0x41, 0x0c, 0xb4, 0x11, 0x77, 0x6d, 0x2c, 0xee, 0xbe, 0x34, 0xb6, 0x1e, 0xf5, + 0xa5, 0x1f, 0xf5, 0x70, 0xa8, 0x05, 0x33, 0x3c, 0xba, 0x53, 0xbb, 0x2f, 0x2b, 0xde, 0x10, 0x6d, + 0xc0, 0x71, 0x76, 0x82, 0x4d, 0xae, 0xd4, 0xec, 0xf0, 0xe5, 0x4c, 0x87, 0x4f, 0x42, 0x0d, 0xdb, + 0x44, 0xe5, 0xb0, 0x36, 0xd1, 0x82, 0x19, 0xae, 0xa7, 0xd4, 0x17, 0x94, 0x15, 0x6f, 0x48, 0xae, + 0x59, 0x37, 0x86, 0x96, 0xed, 0xea, 0x66, 0xbf, 0x55, 0xa5, 0x6b, 0xfe, 0x04, 0x29, 0xb9, 0xc0, + 0x97, 0xe7, 0x84, 0xce, 0xc9, 0x7b, 0x50, 0x16, 0x1a, 0x9e, 0xcb, 0xac, 0xe1, 0x02, 0x27, 0xea, + 0xa3, 0xf3, 0x11, 0x1f, 0x2d, 0xff, 0x55, 0x82, 0xda, 0x0a, 0x39, 0xd2, 0x9a, 0xd5, 0xa7, 0x11, + 0xe5, 0x22, 0x34, 0x6c, 0xdc, 0xb3, 0x6c, 0xad, 0x8b, 0x4d, 0xd7, 0xd6, 0x31, 0xab, 0xce, 0x0b, + 0x4a, 0x9d, 0xcd, 0xde, 0x61, 0x93, 0x04, 0x8c, 0xb8, 0x5d, 0xc7, 0x55, 0x8d, 0x61, 0x77, 0x87, + 0x98, 0x77, 0x8e, 0x81, 0x89, 0x59, 0x6a, 0xdd, 0xe7, 0xa1, 0xe6, 0x83, 0xb9, 0x16, 0xdd, 0xbf, + 0xa0, 0x54, 0xc5, 0xdc, 0x96, 0x85, 0x5e, 0x84, 0x06, 0x95, 0x69, 0x77, 0x60, 0xf5, 0xbb, 0xa4, + 0x92, 0xe5, 0xc1, 0xa6, 0xa6, 0x71, 0xb6, 0xc8, 0x5d, 0x85, 0xa1, 0x1c, 0xfd, 0x53, 0xcc, 0xc3, + 0x8d, 0x80, 0xda, 0xd4, 0x3f, 0xc5, 0x24, 0xd6, 0xd7, 0x49, 0xec, 0xbc, 0x67, 0x69, 0x78, 0xeb, + 0x88, 0x99, 0x46, 0x86, 0x2e, 0xe6, 0x19, 0xa8, 0x88, 0x13, 0xf0, 0x23, 0xf9, 0x13, 0xe8, 0x2e, + 0x34, 0xbc, 0x24, 0xb4, 0xcb, 0x6a, 0xad, 0x42, 0x6a, 0xe6, 0x17, 0x88, 0x7e, 0x8e, 0x52, 0xf7, + 0xd0, 0xe8, 0x50, 0xbe, 0x0b, 0xb5, 0xe0, 0x32, 0xd9, 0x75, 0x33, 0xaa, 0x28, 0x62, 0x82, 0x68, + 0xe3, 0xbd, 0x91, 0x41, 0xee, 0x94, 0x3b, 0x16, 0x6f, 0x28, 0x7f, 0x2e, 0x41, 0x9d, 0x87, 0xec, + 0x4d, 0xd1, 0x65, 0xa7, 0x47, 0x93, 0xe8, 0xd1, 0xe8, 0x6f, 0xf4, 0x3f, 0xe1, 0x16, 0xdd, 0x8b, + 0x89, 0x4e, 0x80, 0x12, 0xa1, 0xd9, 0x71, 0x28, 0x5e, 0x67, 0xa9, 0xed, 0x9f, 0x10, 0x45, 0xe3, + 0x57, 0x43, 0x15, 0xad, 0x05, 0x33, 0xaa, 0xa6, 0xd9, 0xd8, 0x71, 0x38, 0x1f, 0xde, 0x90, 0xac, + 0x3c, 0xc2, 0xb6, 0xe3, 0xa9, 0x7c, 0x5e, 0xf1, 0x86, 0xe8, 0x1d, 0x28, 0x8b, 0x74, 0x3a, 0x9f, + 0x94, 0x42, 0x05, 0xf9, 0xe4, 0xb5, 0xa8, 0xc0, 0x90, 0x7f, 0x9f, 0x83, 0x06, 0x17, 0xd8, 0x2d, + 0x1e, 0x53, 0xc7, 0x1b, 0xdf, 0x2d, 0xa8, 0xed, 0xf8, 0xb6, 0x3f, 0xae, 0xe7, 0x14, 0x74, 0x11, + 0x21, 0x9c, 0x49, 0x06, 0x18, 0x8e, 0xea, 0x85, 0xa9, 0xa2, 0x7a, 0xf1, 0xb0, 0x1e, 0x2c, 0x9e, + 0xe7, 0x95, 0x12, 0xf2, 0x3c, 0xf9, 0x9b, 0x50, 0x0d, 0x10, 0xa0, 0x1e, 0x9a, 0x35, 0xab, 0xb8, + 0xc4, 0xbc, 0x21, 0xba, 0xee, 0xe7, 0x36, 0x4c, 0x54, 0xa7, 0x12, 0x78, 0x89, 0xa4, 0x35, 0xf2, + 0xaf, 0x25, 0x28, 0x71, 0xca, 0xe7, 0xa0, 0xca, 0x9d, 0x0e, 0xcd, 0xfb, 0x18, 0x75, 0xe0, 0x53, + 0x24, 0xf1, 0x7b, 0x7a, 0x5e, 0xe7, 0x14, 0x94, 0x23, 0xfe, 0x66, 0x86, 0x87, 0x05, 0x6f, 0x29, + 0xe0, 0x64, 0xc8, 0x12, 0xf5, 0x2f, 0x5f, 0x49, 0xb4, 0xd1, 0xae, 0xe0, 0x9e, 0xf5, 0x08, 0xdb, + 0x07, 0xd3, 0xb7, 0x33, 0xdf, 0x0e, 0x28, 0x74, 0xc6, 0xfa, 0x50, 0x20, 0xa0, 0xb7, 0x7d, 0x71, + 0xe7, 0x93, 0xba, 0x39, 0x41, 0x0f, 0xc3, 0xd5, 0xd1, 0x17, 0xfb, 0x8f, 0x58, 0x63, 0x36, 0x7c, + 0x94, 0xa3, 0xe6, 0x35, 0x4f, 0xa5, 0xec, 0x90, 0x7f, 0x22, 0xc1, 0xa9, 0x55, 0xec, 0xde, 0x0d, + 0x17, 0xf7, 0xcf, 0x9b, 0x2b, 0x03, 0xda, 0x49, 0x4c, 0x4d, 0x73, 0xeb, 0x6d, 0x28, 0x8b, 0x36, + 0x05, 0x6b, 0x99, 0x8b, 0xb1, 0xfc, 0x5d, 0x09, 0x5a, 0x7c, 0x17, 0xba, 0x27, 0x49, 0xa9, 0x07, + 0xd8, 0xc5, 0xda, 0xd7, 0x5d, 0x37, 0xff, 0x59, 0x82, 0x66, 0xd0, 0xe3, 0x53, 0xa7, 0xfd, 0x26, + 0x14, 0x69, 0x7b, 0x82, 0x73, 0x30, 0x51, 0x59, 0x19, 0x34, 0x71, 0x19, 0x34, 0xcd, 0xdb, 0x12, + 0xc1, 0x89, 0x0f, 0xfd, 0xb0, 0x93, 0x3f, 0x7c, 0xd8, 0xe1, 0x61, 0xd8, 0x1a, 0x11, 0xba, 0xac, + 0x1d, 0xea, 0x4f, 0xc8, 0x5f, 0xe6, 0xa0, 0xe5, 0xd7, 0x23, 0x5f, 0xbb, 0xdf, 0x4f, 0xc9, 0x56, + 0xf3, 0x4f, 0x29, 0x5b, 0x2d, 0x4c, 0xef, 0xeb, 0x8b, 0x49, 0xbe, 0xfe, 0x4f, 0x39, 0x68, 0xf8, + 0x52, 0xdb, 0x18, 0xa8, 0x26, 0x9a, 0x87, 0xd2, 0x70, 0xa0, 0xfa, 0xdd, 0x47, 0x3e, 0x42, 0x9b, + 0x22, 0xcf, 0x09, 0xcb, 0xe9, 0x95, 0xa4, 0x3b, 0x4c, 0xb9, 0x08, 0x25, 0x42, 0x82, 0x94, 0x83, + 0xac, 0xa0, 0xa0, 0x45, 0x3d, 0xcf, 0xad, 0x98, 0xb2, 0x90, 0x7a, 0xfe, 0x0a, 0x20, 0x7e, 0xc3, + 0x5d, 0xdd, 0xec, 0x3a, 0xb8, 0x67, 0x99, 0x1a, 0xbb, 0xfb, 0xa2, 0xd2, 0xe4, 0x2b, 0x1d, 0x73, + 0x93, 0xcd, 0xa3, 0x37, 0xa1, 0xe0, 0x1e, 0x0c, 0x99, 0x17, 0x6f, 0x24, 0x7a, 0x47, 0x9f, 0xaf, + 0xad, 0x83, 0x21, 0x56, 0x28, 0x38, 0x5a, 0x00, 0x20, 0xa4, 0x5c, 0x5b, 0x7d, 0xc4, 0x43, 0x62, + 0x41, 0x09, 0xcc, 0x10, 0x6d, 0xf6, 0x64, 0x38, 0xc3, 0x42, 0x07, 0x1f, 0xca, 0x7f, 0xc8, 0x41, + 0xd3, 0x27, 0xa9, 0x60, 0x67, 0x34, 0x70, 0x53, 0xe5, 0x37, 0xbe, 0x18, 0x9c, 0x94, 0x37, 0xbc, + 0x0f, 0x55, 0x7e, 0x9f, 0x87, 0xd0, 0x07, 0x60, 0x28, 0x6b, 0x63, 0x14, 0xb4, 0xf8, 0x94, 0x14, + 0xb4, 0x74, 0x48, 0x05, 0x95, 0x37, 0x61, 0xde, 0xf3, 0x7b, 0x3e, 0xc0, 0x3a, 0x76, 0xd5, 0x31, + 0x09, 0xc7, 0x39, 0xa8, 0xb2, 0x78, 0xc6, 0x02, 0x39, 0x4b, 0xd5, 0x61, 0x5b, 0x54, 0xb8, 0xf2, + 0xb7, 0xe0, 0x04, 0xf5, 0x1b, 0xd1, 0x56, 0x6e, 0x96, 0xbe, 0xba, 0x2c, 0x0a, 0x01, 0x92, 0xf4, + 0x33, 0xed, 0xae, 0x28, 0xa1, 0x39, 0x79, 0x0d, 0x5e, 0x88, 0xd0, 0x9f, 0x22, 0x2e, 0x90, 0x54, + 0x68, 0x7e, 0x33, 0xfc, 0x99, 0xf8, 0xe8, 0xd1, 0xef, 0xac, 0xe8, 0xdc, 0x76, 0x75, 0x2d, 0xaa, + 0x5f, 0x1a, 0x7a, 0x0f, 0x2a, 0x26, 0xde, 0xef, 0x06, 0x9d, 0x6f, 0x86, 0x06, 0x5d, 0xd9, 0xc4, + 0xfb, 0xf4, 0x97, 0x7c, 0x0f, 0x4e, 0xc6, 0x58, 0x9d, 0xe6, 0xec, 0x7f, 0x94, 0xe0, 0xd4, 0x8a, + 0x6d, 0x0d, 0x3f, 0xd2, 0x6d, 0x77, 0xa4, 0x0e, 0xc2, 0x1f, 0x8a, 0x9e, 0x4d, 0x19, 0xf7, 0x41, + 0x20, 0x0c, 0x33, 0xbf, 0x7c, 0x25, 0x41, 0x5d, 0xe3, 0x4c, 0xf1, 0x43, 0x07, 0x82, 0xf6, 0x3f, + 0xf3, 0x49, 0xcc, 0x73, 0xb8, 0x09, 0xc1, 0x26, 0x4b, 0x96, 0x92, 0xd8, 0xf5, 0xc9, 0x1f, 0xb5, + 0xeb, 0x93, 0x62, 0xf9, 0x85, 0xa7, 0x64, 0xf9, 0x87, 0x2e, 0x43, 0x3e, 0x80, 0x70, 0x47, 0x8e, + 0xba, 0xdc, 0x23, 0xb5, 0xf2, 0x6e, 0x01, 0xf8, 0xdd, 0x29, 0xfe, 0xca, 0x27, 0x0b, 0x99, 0x00, + 0x16, 0xb9, 0x2d, 0xe1, 0x65, 0x69, 0x57, 0x39, 0xd4, 0x2f, 0xb9, 0x0f, 0xed, 0x24, 0x2d, 0x9d, + 0x46, 0xf3, 0xbf, 0xcc, 0x01, 0x74, 0x68, 0x77, 0x68, 0x4b, 0x75, 0xf6, 0x8e, 0x96, 0x51, 0x5e, + 0x80, 0xba, 0xaf, 0x30, 0xbe, 0xbd, 0x07, 0xb5, 0x48, 0x23, 0x26, 0x21, 0x12, 0x5b, 0x02, 0x13, + 0x4b, 0x76, 0x35, 0x4a, 0x27, 0x60, 0x35, 0x4c, 0x29, 0x22, 0x4e, 0x0f, 0x9d, 0x86, 0x8a, 0x6d, + 0xed, 0x77, 0x89, 0x99, 0x69, 0x34, 0xb6, 0x96, 0x95, 0xb2, 0x6d, 0xed, 0x13, 0xe3, 0xd3, 0xd0, + 0x49, 0x98, 0x71, 0x55, 0x67, 0x8f, 0xd0, 0x2f, 0xb1, 0x70, 0x47, 0x86, 0x1d, 0x0d, 0x9d, 0x80, + 0xe2, 0x8e, 0x3e, 0xc0, 0x4e, 0x6b, 0x86, 0x92, 0x64, 0x03, 0xf4, 0x96, 0xf7, 0x9e, 0xa3, 0x9c, + 0xf9, 0x7b, 0x34, 0x7b, 0xd2, 0xf1, 0x95, 0x04, 0xb3, 0xbe, 0xd4, 0xa8, 0x03, 0x22, 0x3e, 0x8d, + 0xfa, 0xb3, 0xdb, 0x96, 0xc6, 0x5c, 0x45, 0x23, 0xe5, 0x13, 0x0b, 0x43, 0x64, 0x5e, 0xcb, 0x47, + 0x19, 0x97, 0x97, 0x93, 0x73, 0x91, 0x43, 0xeb, 0x9a, 0xf7, 0x85, 0xa7, 0x64, 0x5b, 0xfb, 0x1d, + 0x4d, 0x48, 0x83, 0x3d, 0x6b, 0x63, 0x59, 0x28, 0x91, 0xc6, 0x6d, 0xfa, 0xb2, 0xed, 0x02, 0xd4, + 0xb1, 0x6d, 0x5b, 0x76, 0xd7, 0xc0, 0x8e, 0xa3, 0xf6, 0x31, 0x4f, 0xba, 0x6a, 0x74, 0x72, 0x9d, + 0xcd, 0xc9, 0xff, 0xc8, 0x43, 0xc3, 0x3f, 0x8a, 0xf7, 0x5d, 0x47, 0xd7, 0xbc, 0xef, 0x3a, 0xba, + 0x46, 0x9c, 0xb9, 0xcd, 0x5c, 0x61, 0xc0, 0x99, 0xf3, 0x99, 0x8e, 0x46, 0xe2, 0x20, 0x31, 0x30, + 0xd3, 0xd2, 0xb0, 0x7f, 0xb1, 0xe0, 0x4d, 0xf1, 0x7b, 0x0d, 0xe9, 0x47, 0x21, 0x83, 0x7e, 0x14, + 0x33, 0xe8, 0x47, 0x29, 0x41, 0x3f, 0xe6, 0xa1, 0xb4, 0x3d, 0xea, 0xed, 0x61, 0x97, 0xa7, 0x47, + 0x7c, 0x14, 0xd6, 0x9b, 0x72, 0x44, 0x6f, 0x84, 0x7a, 0x54, 0x82, 0xea, 0x71, 0x1a, 0x2a, 0xec, + 0xe3, 0x42, 0xd7, 0x75, 0x68, 0x97, 0x35, 0xaf, 0x94, 0xd9, 0xc4, 0x96, 0x83, 0x6e, 0x78, 0xb5, + 0x43, 0x35, 0xc9, 0xd0, 0xa9, 0xc7, 0x89, 0x68, 0x88, 0x57, 0x39, 0xdc, 0x80, 0xd6, 0x2e, 0x1e, + 0xd9, 0xf4, 0x6d, 0x44, 0x97, 0x3e, 0xde, 0x7d, 0x38, 0xc2, 0xf6, 0x81, 0xba, 0x3d, 0xc0, 0xad, + 0x1a, 0x65, 0x6c, 0x5e, 0xac, 0xaf, 0xa8, 0xae, 0x7a, 0xdf, 0x5b, 0x45, 0x6f, 0xc0, 0x7c, 0x04, + 0x53, 0x37, 0x35, 0xfc, 0x18, 0x6b, 0xad, 0x3a, 0xc5, 0x3b, 0x11, 0xc2, 0xeb, 0xb0, 0x35, 0xf9, + 0x13, 0x40, 0x3e, 0x27, 0xd3, 0xd5, 0x8e, 0x91, 0xab, 0xce, 0x45, 0xaf, 0x5a, 0xfe, 0x8d, 0x04, + 0x73, 0xc1, 0xcd, 0x8e, 0x1a, 0x40, 0xdf, 0x83, 0x2a, 0xeb, 0x59, 0x77, 0x89, 0x01, 0xf3, 0xea, + 0xf1, 0xec, 0x58, 0x19, 0x2b, 0xa0, 0xfb, 0x7e, 0xec, 0x02, 0xd4, 0xf7, 0x2d, 0x7b, 0x4f, 0x37, + 0xfb, 0x5d, 0xc2, 0x99, 0x67, 0x36, 0x35, 0x3e, 0x79, 0x8f, 0xcc, 0xc9, 0x5f, 0x48, 0xb0, 0xf0, + 0xe1, 0x50, 0x53, 0x5d, 0x1c, 0xc8, 0x24, 0xa6, 0x7d, 0x60, 0xf3, 0xa6, 0xf7, 0xc6, 0x25, 0x97, + 0xad, 0xef, 0xca, 0xa0, 0xe5, 0x75, 0x38, 0xa5, 0x60, 0x07, 0x9b, 0x5a, 0x68, 0xf1, 0xa8, 0x5c, + 0xc8, 0x43, 0x68, 0x27, 0x91, 0x9b, 0xe6, 0xee, 0x59, 0x4a, 0xd7, 0xb5, 0x09, 0x59, 0x97, 0x7b, + 0x28, 0x92, 0x49, 0xd0, 0x7d, 0x5c, 0xf9, 0x5f, 0x12, 0xcc, 0xdd, 0xd4, 0xbc, 0xfd, 0x9e, 0x59, + 0xe6, 0x18, 0xcd, 0xac, 0xf2, 0xf1, 0xcc, 0xea, 0x69, 0xb9, 0x1b, 0xee, 0x74, 0xcd, 0x91, 0xe1, + 0x05, 0x13, 0x9b, 0x7e, 0x01, 0xbe, 0xfc, 0x33, 0x09, 0xe6, 0x62, 0x7d, 0x01, 0xd4, 0x00, 0xf8, + 0xd0, 0xec, 0xf1, 0x86, 0x49, 0xf3, 0x18, 0xaa, 0x41, 0xd9, 0x6b, 0x9f, 0x34, 0x25, 0x54, 0x85, + 0x99, 0x2d, 0x8b, 0x42, 0x37, 0x73, 0xa8, 0x09, 0x35, 0x86, 0x38, 0xea, 0xf5, 0xb0, 0xe3, 0x34, + 0xf3, 0x62, 0xe6, 0xae, 0xaa, 0x0f, 0x46, 0x36, 0x6e, 0x16, 0x50, 0x1d, 0x2a, 0x5b, 0x16, 0x7f, + 0x16, 0xd4, 0x2c, 0x22, 0x04, 0x0d, 0xef, 0x8d, 0x10, 0x47, 0x2a, 0x05, 0xe6, 0x3c, 0xb4, 0x99, + 0xcb, 0x3b, 0xc1, 0x0a, 0x9a, 0x94, 0x95, 0xe8, 0x24, 0x1c, 0xff, 0xd0, 0xd4, 0xf0, 0x8e, 0x6e, + 0x62, 0xcd, 0x5f, 0x6a, 0x1e, 0x43, 0xc7, 0x61, 0xb6, 0x63, 0x9a, 0xd8, 0x0e, 0x4c, 0x4a, 0x64, + 0x72, 0x1d, 0xdb, 0x7d, 0x1c, 0x98, 0xcc, 0xa1, 0x39, 0xa8, 0xaf, 0xeb, 0x8f, 0x03, 0x53, 0xf9, + 0xe5, 0xbf, 0x9f, 0x84, 0x0a, 0x71, 0x32, 0xb7, 0x2d, 0xcb, 0xd6, 0xd0, 0x10, 0x10, 0x7d, 0x3a, + 0x67, 0x0c, 0x2d, 0x53, 0xbc, 0x31, 0x45, 0xaf, 0xa5, 0x24, 0x3f, 0x71, 0x50, 0xae, 0x2e, 0xed, + 0x4b, 0x29, 0x18, 0x11, 0x70, 0xf9, 0x18, 0x32, 0xe8, 0x8e, 0xa4, 0x2c, 0xdf, 0xd2, 0x7b, 0x7b, + 0xde, 0xa3, 0x80, 0x31, 0x3b, 0x46, 0x40, 0xbd, 0x1d, 0x23, 0x4f, 0x57, 0xf9, 0x80, 0xbd, 0x6f, + 0xf4, 0xec, 0x45, 0x3e, 0x86, 0x1e, 0xc2, 0x89, 0x55, 0x1c, 0xf0, 0x0f, 0xde, 0x86, 0xcb, 0xe9, + 0x1b, 0xc6, 0x80, 0x0f, 0xb9, 0xe5, 0x1a, 0x14, 0x69, 0x0f, 0x0e, 0x25, 0xb9, 0x90, 0xe0, 0x1f, + 0x31, 0xda, 0x8b, 0xe9, 0x00, 0x82, 0xda, 0x27, 0x30, 0x1b, 0x79, 0x48, 0x8e, 0x5e, 0x4e, 0x40, + 0x4b, 0xfe, 0x4b, 0x40, 0xfb, 0x72, 0x16, 0x50, 0xb1, 0x57, 0x1f, 0x1a, 0xe1, 0x87, 0x77, 0x68, + 0x29, 0x01, 0x3f, 0xf1, 0x11, 0x70, 0xfb, 0xe5, 0x0c, 0x90, 0x62, 0x23, 0x03, 0x9a, 0xd1, 0x87, + 0xcd, 0xe8, 0xf2, 0x58, 0x02, 0x61, 0x75, 0x7b, 0x25, 0x13, 0xac, 0xd8, 0xee, 0x80, 0x2a, 0x41, + 0xec, 0x61, 0x2d, 0xba, 0x9a, 0x4c, 0x26, 0xed, 0xc5, 0x6f, 0xfb, 0x5a, 0x66, 0x78, 0xb1, 0xf5, + 0xb7, 0x59, 0xef, 0x3f, 0xe9, 0x71, 0x2a, 0x7a, 0x3d, 0x99, 0xdc, 0x98, 0x57, 0xb5, 0xed, 0xe5, + 0xc3, 0xa0, 0x08, 0x26, 0x3e, 0xa3, 0x4d, 0xfb, 0x84, 0x07, 0x9e, 0x51, 0xbb, 0xf3, 0xe8, 0xa5, + 0xbf, 0x5c, 0x6d, 0xbf, 0x7e, 0x08, 0x0c, 0xc1, 0x80, 0x15, 0x7d, 0x3a, 0xee, 0x99, 0xe1, 0xb5, + 0x89, 0x5a, 0x73, 0x34, 0x1b, 0xfc, 0x18, 0x66, 0x23, 0xcf, 0x2f, 0x12, 0xad, 0x26, 0xf9, 0x89, + 0x46, 0x7b, 0x5c, 0x58, 0x65, 0x26, 0x19, 0xf9, 0x06, 0x82, 0x52, 0xb4, 0x3f, 0xe1, 0x3b, 0x49, + 0xfb, 0x72, 0x16, 0x50, 0x71, 0x10, 0x87, 0xba, 0xcb, 0xc8, 0x77, 0x04, 0x74, 0x25, 0x99, 0x46, + 0xf2, 0x37, 0x90, 0xf6, 0xab, 0x19, 0xa1, 0xc5, 0xa6, 0x5d, 0x80, 0x55, 0xec, 0xae, 0x63, 0xd7, + 0x26, 0x3a, 0x72, 0x29, 0x51, 0xe4, 0x3e, 0x80, 0xb7, 0xcd, 0x4b, 0x13, 0xe1, 0xc4, 0x06, 0xff, + 0x0f, 0xc8, 0x0b, 0xb1, 0x81, 0xc7, 0x3f, 0x17, 0xc6, 0xb6, 0x5a, 0x59, 0x5f, 0x74, 0xd2, 0xdd, + 0x3c, 0x84, 0xe6, 0xba, 0x6a, 0x92, 0x22, 0xdb, 0xa7, 0x7b, 0x25, 0x91, 0xb1, 0x28, 0x58, 0x8a, + 0xb4, 0x52, 0xa1, 0xc5, 0x61, 0xf6, 0x45, 0x0c, 0x55, 0x85, 0x09, 0xe2, 0xa8, 0x6f, 0xf1, 0xa5, + 0x11, 0x01, 0x4c, 0xf1, 0x2d, 0x63, 0xe0, 0xc5, 0xc6, 0x4f, 0x24, 0xfa, 0x07, 0x85, 0x08, 0xc0, + 0x03, 0xdd, 0xdd, 0xdd, 0x18, 0xa8, 0xa6, 0x93, 0x85, 0x05, 0x0a, 0x78, 0x08, 0x16, 0x38, 0xbc, + 0x60, 0x41, 0x83, 0x7a, 0xa8, 0x93, 0x89, 0x92, 0x5e, 0xf0, 0x24, 0xf5, 0x52, 0xdb, 0x4b, 0x93, + 0x01, 0xc5, 0x2e, 0xbb, 0x50, 0xf7, 0xf4, 0x95, 0x09, 0xf7, 0xe5, 0x34, 0x4e, 0x7d, 0x98, 0x14, + 0x73, 0x4b, 0x06, 0x0d, 0x9a, 0x5b, 0xbc, 0x51, 0x83, 0xb2, 0x35, 0xf8, 0xc6, 0x99, 0x5b, 0x7a, + 0xf7, 0x87, 0xf9, 0x93, 0x48, 0x53, 0x34, 0xd9, 0x59, 0x25, 0xf6, 0x78, 0x13, 0xfd, 0x49, 0x4a, + 0x8f, 0x55, 0x3e, 0x86, 0x1e, 0x40, 0x89, 0x55, 0x5e, 0xe8, 0xc5, 0xf1, 0x45, 0x19, 0xa7, 0x7e, + 0x71, 0x02, 0x94, 0x20, 0xbc, 0x07, 0x27, 0x53, 0x4a, 0xb2, 0xc4, 0x38, 0x37, 0xbe, 0x7c, 0x9b, + 0x64, 0xe5, 0x2a, 0xa0, 0xf8, 0x1f, 0x00, 0x12, 0xaf, 0x29, 0xf5, 0x7f, 0x02, 0x19, 0xb6, 0x88, + 0xbf, 0xed, 0x4f, 0xdc, 0x22, 0xf5, 0x2f, 0x00, 0x93, 0xb6, 0xb8, 0x0f, 0xe0, 0x17, 0x5e, 0x89, + 0xf7, 0x11, 0xab, 0xcb, 0x26, 0x90, 0x5c, 0xfe, 0x77, 0x09, 0xca, 0xde, 0x7b, 0x99, 0xe7, 0x90, + 0xdc, 0x3f, 0x87, 0x6c, 0xfb, 0x63, 0x98, 0x8d, 0x3c, 0xbc, 0x4f, 0x34, 0x9e, 0xe4, 0xc7, 0xf9, + 0x93, 0x6e, 0xe8, 0x01, 0xff, 0x7b, 0xb2, 0x08, 0xbc, 0x2f, 0xa5, 0x65, 0xec, 0xd1, 0x98, 0x3b, + 0x81, 0xf0, 0x33, 0x8f, 0xb0, 0xf7, 0x00, 0x02, 0x11, 0x70, 0xfc, 0x47, 0x4c, 0xe2, 0xd4, 0x27, + 0x31, 0xbc, 0x7e, 0x48, 0xbf, 0x31, 0x81, 0x9c, 0x43, 0xac, 0x2b, 0xda, 0xe6, 0x48, 0xb1, 0xae, + 0x94, 0xe6, 0x4a, 0xa2, 0x9f, 0x4d, 0xef, 0x9d, 0x3c, 0x13, 0x7b, 0xbb, 0x75, 0xfd, 0x1b, 0xaf, + 0xf7, 0x75, 0x77, 0x77, 0xb4, 0x4d, 0x56, 0xae, 0x31, 0xd0, 0x57, 0x75, 0x8b, 0xff, 0xba, 0xe6, + 0x29, 0xfa, 0x35, 0x8a, 0x7d, 0x8d, 0xec, 0x31, 0xdc, 0xde, 0x2e, 0xd1, 0xd1, 0xf5, 0xff, 0x06, + 0x00, 0x00, 0xff, 0xff, 0xe7, 0xa4, 0x10, 0xe3, 0x01, 0x40, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -4256,6 +4339,7 @@ type DataCoordClient interface { UpdateSegmentStatistics(ctx context.Context, in *UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) ReleaseSegmentLock(ctx context.Context, in *ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + AddSegment(ctx context.Context, in *AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } type dataCoordClient struct { @@ -4509,6 +4593,15 @@ func (c *dataCoordClient) ReleaseSegmentLock(ctx context.Context, in *ReleaseSeg return out, nil } +func (c *dataCoordClient) AddSegment(ctx context.Context, in *AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/AddSegment", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DataCoordServer is the server API for DataCoord service. type DataCoordServer interface { GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) @@ -4540,6 +4633,7 @@ type DataCoordServer interface { UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) AcquireSegmentLock(context.Context, *AcquireSegmentLockRequest) (*commonpb.Status, error) ReleaseSegmentLock(context.Context, *ReleaseSegmentLockRequest) (*commonpb.Status, error) + AddSegment(context.Context, *AddSegmentRequest) (*commonpb.Status, error) } // UnimplementedDataCoordServer can be embedded to have forward compatible implementations. @@ -4627,6 +4721,9 @@ func (*UnimplementedDataCoordServer) AcquireSegmentLock(ctx context.Context, req func (*UnimplementedDataCoordServer) ReleaseSegmentLock(ctx context.Context, req *ReleaseSegmentLockRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method ReleaseSegmentLock not implemented") } +func (*UnimplementedDataCoordServer) AddSegment(ctx context.Context, req *AddSegmentRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AddSegment not implemented") +} func RegisterDataCoordServer(s *grpc.Server, srv DataCoordServer) { s.RegisterService(&_DataCoord_serviceDesc, srv) @@ -5118,6 +5215,24 @@ func _DataCoord_ReleaseSegmentLock_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _DataCoord_AddSegment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AddSegmentRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).AddSegment(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/AddSegment", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).AddSegment(ctx, req.(*AddSegmentRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _DataCoord_serviceDesc = grpc.ServiceDesc{ ServiceName: "milvus.proto.data.DataCoord", HandlerType: (*DataCoordServer)(nil), @@ -5230,6 +5345,10 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{ MethodName: "ReleaseSegmentLock", Handler: _DataCoord_ReleaseSegmentLock_Handler, }, + { + MethodName: "AddSegment", + Handler: _DataCoord_AddSegment_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "data_coord.proto", @@ -5249,6 +5368,7 @@ type DataNodeClient interface { // 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) + AddSegment(ctx context.Context, in *AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } type dataNodeClient struct { @@ -5331,6 +5451,15 @@ func (c *dataNodeClient) ResendSegmentStats(ctx context.Context, in *ResendSegme return out, nil } +func (c *dataNodeClient) AddSegment(ctx context.Context, in *AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataNode/AddSegment", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DataNodeServer is the server API for DataNode service. type DataNodeServer interface { GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) @@ -5343,6 +5472,7 @@ type DataNodeServer interface { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load Import(context.Context, *ImportTaskRequest) (*commonpb.Status, error) ResendSegmentStats(context.Context, *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error) + AddSegment(context.Context, *AddSegmentRequest) (*commonpb.Status, error) } // UnimplementedDataNodeServer can be embedded to have forward compatible implementations. @@ -5373,6 +5503,9 @@ func (*UnimplementedDataNodeServer) Import(ctx context.Context, req *ImportTaskR func (*UnimplementedDataNodeServer) ResendSegmentStats(ctx context.Context, req *ResendSegmentStatsRequest) (*ResendSegmentStatsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResendSegmentStats not implemented") } +func (*UnimplementedDataNodeServer) AddSegment(ctx context.Context, req *AddSegmentRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AddSegment not implemented") +} func RegisterDataNodeServer(s *grpc.Server, srv DataNodeServer) { s.RegisterService(&_DataNode_serviceDesc, srv) @@ -5522,6 +5655,24 @@ func _DataNode_ResendSegmentStats_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _DataNode_AddSegment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AddSegmentRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).AddSegment(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataNode/AddSegment", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).AddSegment(ctx, req.(*AddSegmentRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _DataNode_serviceDesc = grpc.ServiceDesc{ ServiceName: "milvus.proto.data.DataNode", HandlerType: (*DataNodeServer)(nil), @@ -5558,6 +5709,10 @@ var _DataNode_serviceDesc = grpc.ServiceDesc{ MethodName: "ResendSegmentStats", Handler: _DataNode_ResendSegmentStats_Handler, }, + { + MethodName: "AddSegment", + Handler: _DataNode_AddSegment_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "data_coord.proto", diff --git a/internal/proxy/data_coord_mock_test.go b/internal/proxy/data_coord_mock_test.go index 8fff6192a1..adfc77b25b 100644 --- a/internal/proxy/data_coord_mock_test.go +++ b/internal/proxy/data_coord_mock_test.go @@ -114,6 +114,10 @@ func (coord *DataCoordMock) Flush(ctx context.Context, req *datapb.FlushRequest) panic("implement me") } +func (coord *DataCoordMock) AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) { + panic("implement me") +} + func (coord *DataCoordMock) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) { panic("implement me") } diff --git a/internal/querynode/segment_loader_test.go b/internal/querynode/segment_loader_test.go index 7629ff8807..68e4aa3fe9 100644 --- a/internal/querynode/segment_loader_test.go +++ b/internal/querynode/segment_loader_test.go @@ -23,9 +23,6 @@ import ( "runtime" "testing" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/proto/commonpb" @@ -34,6 +31,8 @@ import ( "github.com/milvus-io/milvus/internal/proto/schemapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/funcutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" ) func TestSegmentLoader_loadSegment(t *testing.T) { diff --git a/internal/types/types.go b/internal/types/types.go index 53ef200a58..012eff2ed4 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -81,6 +81,9 @@ type DataNode interface { // ResendSegmentStats resend un-flushed segment stats back upstream to DataCoord by resending DataNode time tick message. // It returns a list of segments to be sent. ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) + + // AddSegment puts the given segment to current DataNode's flow graph. + AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) } // DataNodeComponent is used by grpc server of DataNode @@ -284,6 +287,10 @@ type DataCoord interface { AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) + + // AddSegment looks for the right DataNode given channel name, and triggers AddSegment call on that DataNode to + // add the segment into this DataNode. + AddSegment(ctx context.Context, req *datapb.AddSegmentRequest) (*commonpb.Status, error) } // DataCoordComponent defines the interface of DataCoord component. diff --git a/internal/util/mock/grpc_datacoord_client.go b/internal/util/mock/grpc_datacoord_client.go index d0585c1315..aa100d4c67 100644 --- a/internal/util/mock/grpc_datacoord_client.go +++ b/internal/util/mock/grpc_datacoord_client.go @@ -140,3 +140,7 @@ func (m *GrpcDataCoordClient) AcquireSegmentLock(ctx context.Context, req *datap func (m *GrpcDataCoordClient) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err } + +func (m *GrpcDataCoordClient) AddSegment(ctx context.Context, in *datapb.AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} diff --git a/internal/util/mock/grpc_datanode_client.go b/internal/util/mock/grpc_datanode_client.go index e25ee429c1..ff28c81f05 100644 --- a/internal/util/mock/grpc_datanode_client.go +++ b/internal/util/mock/grpc_datanode_client.go @@ -64,3 +64,7 @@ func (m *GrpcDataNodeClient) Import(ctx context.Context, req *datapb.ImportTaskR func (m *GrpcDataNodeClient) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest, opts ...grpc.CallOption) (*datapb.ResendSegmentStatsResponse, error) { return &datapb.ResendSegmentStatsResponse{}, m.Err } + +func (m *GrpcDataNodeClient) AddSegment(ctx context.Context, in *datapb.AddSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +}