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

/kind feature

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -776,14 +776,14 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan
} }
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments // Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTask) (*commonpb.Status, error) { func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
log.Info("receive import request") log.Info("receive import request")
if !node.isHealthy() { if !node.isHealthy() {
log.Warn("DataNode.Import failed", log.Warn("DataNode.Import failed",
zap.Int64("collection ID", req.GetCollectionId()), zap.Int64("collection ID", req.GetImportTask().GetCollectionId()),
zap.Int64("partition ID", req.GetPartitionId()), zap.Int64("partition ID", req.GetImportTask().GetPartitionId()),
zap.Int64("taskID", req.GetTaskId()), zap.Int64("taskID", req.GetImportTask().GetTaskId()),
zap.Error(errDataNodeIsUnhealthy(Params.DataNodeCfg.NodeID))) zap.Error(errDataNodeIsUnhealthy(Params.DataNodeCfg.NodeID)))
return &commonpb.Status{ return &commonpb.Status{

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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