mirror of https://github.com/milvus-io/milvus.git
parent
7f044fff82
commit
9828a52999
|
@ -21,7 +21,7 @@ msgChannel:
|
|||
proxyTimeTick: "proxyTimeTick"
|
||||
writeNodeTimeTick: "writeNodeTimeTick" # GOOSE TODO: remove this
|
||||
dataNodeTimeTick: "dataNodeTimeTick"
|
||||
queryNodeTimeTick: "queryNodeTimeTick"
|
||||
queryTimeTick: "queryTimeTick"
|
||||
dataNodeSegStatistics: "dataNodeSegStatistics"
|
||||
# old name: statsChannels: "statistic"
|
||||
queryNodeStats: "query-node-stats"
|
||||
|
|
|
@ -52,8 +52,6 @@ proxyNode:
|
|||
port: 19530
|
||||
|
||||
queryNode:
|
||||
ip: localhost
|
||||
port: 20010
|
||||
gracefulTime: 5000 #ms
|
||||
|
||||
indexBuilder:
|
||||
|
|
|
@ -78,9 +78,9 @@ In order to boost throughput, we model Milvus as a stream-driven system.
|
|||
|
||||
```go
|
||||
type Component interface {
|
||||
Init()
|
||||
Start()
|
||||
Stop()
|
||||
Init() error
|
||||
Start() error
|
||||
Stop() error
|
||||
GetComponentStates() (ComponentStates, error)
|
||||
GetTimeTickChannel() (string, error)
|
||||
GetStatisticsChannel() (string, error)
|
||||
|
|
|
@ -11,15 +11,15 @@ type Client struct {
|
|||
// GOOSE TODO: add DataNodeClient
|
||||
}
|
||||
|
||||
func (c *Client) Init() {
|
||||
func (c *Client) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (c *Client) Start() {
|
||||
func (c *Client) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (c *Client) Stop() {
|
||||
func (c *Client) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ func (g Client) GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexS
|
|||
return g.grpcClient.GetIndexStates(ctx, req)
|
||||
}
|
||||
|
||||
func (g Client) GetIndexFilePaths(req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
func (g Client) GetIndexFilePaths(req *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
|
||||
ctx := context.TODO()
|
||||
|
||||
|
|
|
@ -28,8 +28,9 @@ type Server struct {
|
|||
loopWg sync.WaitGroup
|
||||
}
|
||||
|
||||
func (s *Server) Init() {
|
||||
func (s *Server) Init() error {
|
||||
indexservice.Params.Init()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) Start() error {
|
||||
|
@ -37,8 +38,9 @@ func (s *Server) Start() error {
|
|||
return s.startIndexServer()
|
||||
}
|
||||
|
||||
func (s *Server) Stop() {
|
||||
func (s *Server) Stop() error {
|
||||
s.loopWg.Wait()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) GetComponentStates() (*internalpb2.ComponentStates, error) {
|
||||
|
@ -69,7 +71,7 @@ func (s *Server) GetIndexStates(ctx context.Context, req *indexpb.IndexStatesReq
|
|||
return s.server.GetIndexStates(req)
|
||||
}
|
||||
|
||||
func (s *Server) GetIndexFilePaths(ctx context.Context, req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
func (s *Server) GetIndexFilePaths(ctx context.Context, req *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
|
||||
return s.server.GetIndexFilePaths(req)
|
||||
}
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
|
@ -37,6 +38,7 @@ func TestGrpcService(t *testing.T) {
|
|||
|
||||
cms.Params.MaxPartitionNum = 64
|
||||
cms.Params.DefaultPartitionName = "_default"
|
||||
cms.Params.DefaultIndexName = "_default"
|
||||
|
||||
t.Logf("master service port = %d", cms.Params.Port)
|
||||
|
||||
|
@ -84,6 +86,18 @@ func TestGrpcService(t *testing.T) {
|
|||
return nil
|
||||
}
|
||||
|
||||
core.DataServiceSegmentChan = make(chan *datapb.SegmentInfo, 1024)
|
||||
|
||||
core.GetBinlogFilePathsFromDataServiceReq = func(segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error) {
|
||||
return []string{"file1", "file2", "file3"}, nil
|
||||
}
|
||||
|
||||
binlogPathArray := make([]string, 0, 16)
|
||||
core.BuildIndexReq = func(binlog []string, typeParams []*commonpb.KeyValuePair, indexParams []*commonpb.KeyValuePair) (typeutil.UniqueID, error) {
|
||||
binlogPathArray = append(binlogPathArray, binlog...)
|
||||
return 2000, nil
|
||||
}
|
||||
|
||||
err = svr.Start()
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
@ -275,6 +289,107 @@ func TestGrpcService(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.Equal(t, len(rsp.PartitionNames), 2)
|
||||
assert.Equal(t, len(rsp.PartitionIDs), 2)
|
||||
})
|
||||
|
||||
t.Run("show segment", func(t *testing.T) {
|
||||
coll, err := core.MetaTable.GetCollectionByName("testColl")
|
||||
assert.Nil(t, err)
|
||||
partID := coll.PartitionIDs[1]
|
||||
part, err := core.MetaTable.GetPartitionByID(partID)
|
||||
assert.Nil(t, err)
|
||||
assert.Zero(t, len(part.SegmentIDs))
|
||||
seg := &datapb.SegmentInfo{
|
||||
SegmentID: 1000,
|
||||
CollectionID: coll.ID,
|
||||
PartitionID: part.PartitionID,
|
||||
}
|
||||
core.DataServiceSegmentChan <- seg
|
||||
time.Sleep(time.Millisecond * 100)
|
||||
part, err = core.MetaTable.GetPartitionByID(partID)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(part.SegmentIDs), 1)
|
||||
|
||||
req := &milvuspb.ShowSegmentRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: 111, //TODO show segment request msg type
|
||||
MsgID: 111,
|
||||
Timestamp: 111,
|
||||
SourceID: 111,
|
||||
},
|
||||
CollectionID: coll.ID,
|
||||
PartitionID: partID,
|
||||
}
|
||||
rsp, err := cli.ShowSegments(req)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.Equal(t, rsp.SegmentIDs[0], int64(1000))
|
||||
assert.Equal(t, len(rsp.SegmentIDs), 1)
|
||||
})
|
||||
|
||||
t.Run("create index", func(t *testing.T) {
|
||||
req := &milvuspb.CreateIndexRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kCreateIndex,
|
||||
MsgID: 112,
|
||||
Timestamp: 112,
|
||||
SourceID: 112,
|
||||
},
|
||||
DbName: "",
|
||||
CollectionName: "testColl",
|
||||
FieldName: "vector",
|
||||
ExtraParams: nil,
|
||||
}
|
||||
rsp, err := cli.CreateIndex(req)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, rsp.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.Equal(t, 3, len(binlogPathArray))
|
||||
assert.ElementsMatch(t, binlogPathArray, []string{"file1", "file2", "file3"})
|
||||
|
||||
req.FieldName = "no field"
|
||||
rsp, err = cli.CreateIndex(req)
|
||||
assert.Nil(t, err)
|
||||
assert.NotEqual(t, rsp.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
})
|
||||
|
||||
t.Run("describe segment", func(t *testing.T) {
|
||||
coll, err := core.MetaTable.GetCollectionByName("testColl")
|
||||
assert.Nil(t, err)
|
||||
|
||||
req := &milvuspb.DescribeSegmentRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: 113, //TODO, describe segment request msg type
|
||||
MsgID: 113,
|
||||
Timestamp: 113,
|
||||
SourceID: 113,
|
||||
},
|
||||
CollectionID: coll.ID,
|
||||
SegmentID: 1000,
|
||||
}
|
||||
rsp, err := cli.DescribeSegment(req)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
t.Logf("index id = %d", rsp.IndexID)
|
||||
})
|
||||
|
||||
t.Run("describe index", func(t *testing.T) {
|
||||
req := &milvuspb.DescribeIndexRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kDescribeIndex,
|
||||
MsgID: 114,
|
||||
Timestamp: 114,
|
||||
SourceID: 114,
|
||||
},
|
||||
DbName: "",
|
||||
CollectionName: "testColl",
|
||||
FieldName: "vector",
|
||||
IndexName: "",
|
||||
}
|
||||
rsp, err := cli.DescribeIndex(req)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, rsp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.Equal(t, len(rsp.IndexDescriptions), 1)
|
||||
assert.Equal(t, rsp.IndexDescriptions[0].IndexName, cms.Params.DefaultIndexName)
|
||||
})
|
||||
|
||||
t.Run("drop partition", func(t *testing.T) {
|
||||
|
|
|
@ -13,15 +13,15 @@ type Client struct {
|
|||
grpcClient querypb.QueryNodeClient
|
||||
}
|
||||
|
||||
func (c *Client) Init() {
|
||||
func (c *Client) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (c *Client) Start() {
|
||||
func (c *Client) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (c *Client) Stop() {
|
||||
func (c *Client) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -9,15 +9,15 @@ type Client struct {
|
|||
grpcClient querypb.QueryServiceClient
|
||||
}
|
||||
|
||||
func (c *Client) Init() {
|
||||
func (c *Client) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (c *Client) Start() {
|
||||
func (c *Client) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (c *Client) Stop() {
|
||||
func (c *Client) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -13,15 +13,15 @@ type Server struct {
|
|||
queryService queryServiceImpl.Interface
|
||||
}
|
||||
|
||||
func (s *Server) Init() {
|
||||
func (s *Server) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (s *Server) Start() {
|
||||
func (s *Server) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (s *Server) Stop() {
|
||||
func (s *Server) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -133,13 +133,13 @@ func (c *Client) GetIndexStates(indexIDs []UniqueID) (*indexpb.IndexStatesRespon
|
|||
return response, err
|
||||
}
|
||||
|
||||
func (c *Client) GetIndexFilePaths(indexID UniqueID) ([]string, error) {
|
||||
func (c *Client) GetIndexFilePaths(indexIDs []UniqueID) ([][]string, error) {
|
||||
if c.tryConnect() != nil {
|
||||
panic("GetIndexFilePaths: failed to connect index builder")
|
||||
}
|
||||
ctx := context.TODO()
|
||||
request := &indexpb.IndexFilePathRequest{
|
||||
IndexID: indexID,
|
||||
request := &indexpb.IndexFilePathsRequest{
|
||||
IndexIDs: indexIDs,
|
||||
}
|
||||
|
||||
response, err := c.client.GetIndexFilePaths(ctx, request)
|
||||
|
@ -147,5 +147,15 @@ func (c *Client) GetIndexFilePaths(indexID UniqueID) ([]string, error) {
|
|||
return nil, err
|
||||
}
|
||||
|
||||
return response.IndexFilePaths, nil
|
||||
var filePaths [][]string
|
||||
for _, indexID := range indexIDs {
|
||||
for _, filePathInfo := range response.FilePaths {
|
||||
if indexID == filePathInfo.IndexID {
|
||||
filePaths = append(filePaths, filePathInfo.IndexFilePaths)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return filePaths, nil
|
||||
}
|
||||
|
|
|
@ -74,16 +74,25 @@ func (b *Builder) GetIndexStates(ctx context.Context, request *indexpb.IndexStat
|
|||
return ret, nil
|
||||
}
|
||||
|
||||
func (b *Builder) GetIndexFilePaths(ctx context.Context, request *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
func (b *Builder) GetIndexFilePaths(ctx context.Context, request *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
ret := &indexpb.IndexFilePathsResponse{
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
|
||||
IndexID: request.IndexID,
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
|
||||
}
|
||||
filePaths, err := b.metaTable.GetIndexFilePaths(request.IndexID)
|
||||
if err != nil {
|
||||
ret.Status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
|
||||
ret.Status.Reason = err.Error()
|
||||
var filePathInfos []*indexpb.IndexFilePathInfo
|
||||
for _, indexID := range request.IndexIDs {
|
||||
filePathInfo := &indexpb.IndexFilePathInfo{
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
|
||||
IndexID: indexID,
|
||||
}
|
||||
|
||||
filePaths, err := b.metaTable.GetIndexFilePaths(indexID)
|
||||
if err != nil {
|
||||
filePathInfo.Status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
|
||||
filePathInfo.Status.Reason = err.Error()
|
||||
}
|
||||
filePathInfo.IndexFilePaths = filePaths
|
||||
filePathInfos = append(filePathInfos, filePathInfo)
|
||||
}
|
||||
ret.IndexFilePaths = filePaths
|
||||
ret.FilePaths = filePathInfos
|
||||
return ret, nil
|
||||
}
|
||||
|
|
|
@ -45,15 +45,15 @@ type IndexNode struct {
|
|||
//serviceClient indexservice.Interface // method factory
|
||||
}
|
||||
|
||||
func (i *IndexNode) Init() {
|
||||
func (i *IndexNode) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (i *IndexNode) Start() {
|
||||
func (i *IndexNode) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (i *IndexNode) Stop() {
|
||||
func (i *IndexNode) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -124,8 +124,7 @@ func TestBuilder_GRPC(t *testing.T) {
|
|||
assert.Equal(t, commonpb.IndexState_INPROGRESS, description.States[0].State)
|
||||
assert.Equal(t, indexID, description.States[0].IndexID)
|
||||
|
||||
indexDataPaths, err := buildClient.GetIndexFilePaths(indexID)
|
||||
indexDataPaths, err := buildClient.GetIndexFilePaths([]UniqueID{indexID})
|
||||
assert.Nil(t, err)
|
||||
assert.Nil(t, indexDataPaths)
|
||||
|
||||
assert.Nil(t, indexDataPaths[0])
|
||||
}
|
||||
|
|
|
@ -46,15 +46,15 @@ type IndexService struct {
|
|||
type UniqueID = typeutil.UniqueID
|
||||
type Timestamp = typeutil.Timestamp
|
||||
|
||||
func (i *IndexService) Init() {
|
||||
func (i *IndexService) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (i *IndexService) Start() {
|
||||
func (i *IndexService) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (i *IndexService) Stop() {
|
||||
func (i *IndexService) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
@ -149,7 +149,7 @@ func (i *IndexService) GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb
|
|||
return ret, nil
|
||||
}
|
||||
|
||||
func (i *IndexService) GetIndexFilePaths(req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
func (i *IndexService) GetIndexFilePaths(req *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error) {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -13,6 +13,6 @@ type Interface interface {
|
|||
RegisterNode(req *indexpb.RegisterNodeRequest) (*indexpb.RegisterNodeResponse, error)
|
||||
BuildIndex(req *indexpb.BuildIndexRequest) (*indexpb.BuildIndexResponse, error)
|
||||
GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error)
|
||||
GetIndexFilePaths(req *indexpb.IndexFilePathRequest) (*indexpb.IndexFilePathsResponse, error)
|
||||
GetIndexFilePaths(req *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error)
|
||||
NotifyBuildIndex(nty *indexpb.BuildIndexNotification) (*commonpb.Status, error)
|
||||
}
|
||||
|
|
|
@ -65,7 +65,7 @@ func (m *MockWriteNodeClient) GetInsertBinlogPaths(segmentID UniqueID) (map[Uniq
|
|||
type BuildIndexClient interface {
|
||||
BuildIndex(columnDataPaths []string, typeParams map[string]string, indexParams map[string]string) (UniqueID, error)
|
||||
GetIndexStates(indexIDs []UniqueID) (*indexpb.IndexStatesResponse, error)
|
||||
GetIndexFilePaths(indexID UniqueID) ([]string, error)
|
||||
GetIndexFilePaths(indexID []UniqueID) ([][]string, error)
|
||||
}
|
||||
|
||||
type MockBuildIndexClient struct {
|
||||
|
@ -107,8 +107,8 @@ func (m *MockBuildIndexClient) GetIndexStates(indexIDs []UniqueID) (*indexpb.Ind
|
|||
return ret, nil
|
||||
}
|
||||
|
||||
func (m *MockBuildIndexClient) GetIndexFilePaths(indexID UniqueID) ([]string, error) {
|
||||
return []string{"/binlog/index/file_1", "/binlog/index/file_2", "/binlog/index/file_3"}, nil
|
||||
func (m *MockBuildIndexClient) GetIndexFilePaths(indexIDs []UniqueID) ([][]string, error) {
|
||||
return [][]string{{"/binlog/index/file_1", "/binlog/index/file_2", "/binlog/index/file_3"}}, nil
|
||||
}
|
||||
|
||||
type LoadIndexClient interface {
|
||||
|
|
|
@ -116,10 +116,11 @@ func (scheduler *IndexBuildScheduler) describe() error {
|
|||
}
|
||||
if description.States[0].State == commonpb.IndexState_FINISHED {
|
||||
log.Printf("build index for segment %d field %d is finished", indexBuildInfo.segmentID, indexBuildInfo.fieldID)
|
||||
filePaths, err := scheduler.client.GetIndexFilePaths(indexID)
|
||||
filesPaths, err := scheduler.client.GetIndexFilePaths([]UniqueID{indexID})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
filePaths := filesPaths[0]
|
||||
|
||||
//TODO: remove fileName
|
||||
var fieldName string
|
||||
|
|
|
@ -14,6 +14,7 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
|
@ -134,6 +135,18 @@ type Core struct {
|
|||
//setMsgStreams, send drop partition into dd channel
|
||||
DdDropPartitionReq func(req *internalpb2.DropPartitionRequest) error
|
||||
|
||||
//setMsgStreams segment channel, receive segment info from data service, if master create segment
|
||||
DataServiceSegmentChan chan *datapb.SegmentInfo
|
||||
|
||||
//TODO,get binlog file path from data service,
|
||||
GetBinlogFilePathsFromDataServiceReq func(segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error)
|
||||
|
||||
//TODO, call index builder's client to build index, return build id
|
||||
BuildIndexReq func(binlog []string, typeParams []*commonpb.KeyValuePair, indexParams []*commonpb.KeyValuePair) (typeutil.UniqueID, error)
|
||||
|
||||
// put create index task into this chan
|
||||
indexTaskQueue chan *CreateIndexTask
|
||||
|
||||
//dd request scheduler
|
||||
ddReqQueue chan reqTask //dd request will be push into this chan
|
||||
lastDdTimeStamp typeutil.Timestamp
|
||||
|
@ -202,6 +215,18 @@ func (c *Core) checkInit() error {
|
|||
if c.DdDropPartitionReq == nil {
|
||||
return errors.Errorf("DdDropPartitionReq is nil")
|
||||
}
|
||||
if c.DataServiceSegmentChan == nil {
|
||||
return errors.Errorf("DataServiceSegmentChan is nil")
|
||||
}
|
||||
if c.GetBinlogFilePathsFromDataServiceReq == nil {
|
||||
return errors.Errorf("GetBinlogFilePathsFromDataServiceReq is nil")
|
||||
}
|
||||
if c.BuildIndexReq == nil {
|
||||
return errors.Errorf("BuildIndexReq is nil")
|
||||
}
|
||||
if c.indexTaskQueue == nil {
|
||||
return errors.Errorf("indexTaskQueue is nil")
|
||||
}
|
||||
log.Printf("master node id = %d\n", Params.NodeID)
|
||||
return nil
|
||||
}
|
||||
|
@ -255,6 +280,47 @@ func (c *Core) startTimeTickLoop() {
|
|||
}
|
||||
}
|
||||
|
||||
//data service send segment info to master when create segment
|
||||
func (c *Core) startDataServiceSegmentLoop() {
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
log.Printf("close data service segment loop")
|
||||
return
|
||||
case seg, ok := <-c.DataServiceSegmentChan:
|
||||
if !ok {
|
||||
log.Printf("data service segment is closed, exit loop")
|
||||
return
|
||||
}
|
||||
if seg == nil {
|
||||
log.Printf("segment from data service is nill")
|
||||
} else if err := c.MetaTable.AddSegment(seg); err != nil {
|
||||
//what if master add segment failed, but data service success?
|
||||
log.Printf("add segment info meta table failed ")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//create index loop
|
||||
func (c *Core) startCreateIndexLoop() {
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
log.Printf("close create index loop")
|
||||
return
|
||||
case t, ok := <-c.indexTaskQueue:
|
||||
if !ok {
|
||||
log.Printf("index task chan is close, exit loop")
|
||||
return
|
||||
}
|
||||
if err := t.BuildIndex(); err != nil {
|
||||
log.Printf("create index failed, error = %s", err.Error())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Core) setMsgStreams() error {
|
||||
//proxy time tick stream,
|
||||
proxyTimeTickStream := pulsarms.NewPulsarMsgStream(c.ctx, 1024)
|
||||
|
@ -405,6 +471,7 @@ func (c *Core) setMsgStreams() error {
|
|||
dataServiceStream.SetPulsarClient(Params.PulsarAddress)
|
||||
dataServiceStream.CreatePulsarConsumers([]string{Params.DataServiceSegmentChannel}, Params.MsgChannelSubName, util.NewUnmarshalDispatcher(), 1024)
|
||||
dataServiceStream.Start()
|
||||
c.DataServiceSegmentChan = make(chan *datapb.SegmentInfo, 1024)
|
||||
|
||||
// receive segment info from msg stream
|
||||
go func() {
|
||||
|
@ -420,9 +487,7 @@ func (c *Core) setMsgStreams() error {
|
|||
for _, segm := range segMsg.Msgs {
|
||||
segInfoMsg, ok := segm.(*ms.SegmentInfoMsg)
|
||||
if ok {
|
||||
if err := c.MetaTable.AddSegment(segInfoMsg.Segment); err != nil {
|
||||
log.Printf("create segment failed, segmentid = %d,colleciont id = %d, error = %s", segInfoMsg.Segment.SegmentID, segInfoMsg.Segment.CollectionID, err.Error())
|
||||
}
|
||||
c.DataServiceSegmentChan <- segInfoMsg.Segment
|
||||
}
|
||||
//TODO, if data node flush
|
||||
}
|
||||
|
@ -457,6 +522,7 @@ func (c *Core) Init(params *InitParams) error {
|
|||
return
|
||||
}
|
||||
c.ddReqQueue = make(chan reqTask, 1024)
|
||||
c.indexTaskQueue = make(chan *CreateIndexTask, 1024)
|
||||
initError = c.setMsgStreams()
|
||||
c.isInit.Store(true)
|
||||
})
|
||||
|
@ -474,6 +540,8 @@ func (c *Core) Start() error {
|
|||
c.startOnce.Do(func() {
|
||||
go c.startDdScheduler()
|
||||
go c.startTimeTickLoop()
|
||||
go c.startDataServiceSegmentLoop()
|
||||
go c.startCreateIndexLoop()
|
||||
c.stateCode.Store(internalpb2.StateCode_HEALTHY)
|
||||
})
|
||||
return nil
|
||||
|
@ -760,24 +828,116 @@ func (c *Core) ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.Show
|
|||
return t.Rsp, nil
|
||||
}
|
||||
|
||||
//TODO
|
||||
func (c *Core) CreateIndex(in *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
|
||||
return nil, nil
|
||||
t := &CreateIndexReqTask{
|
||||
baseReqTask: baseReqTask{
|
||||
cv: make(chan error),
|
||||
core: c,
|
||||
},
|
||||
Req: in,
|
||||
}
|
||||
c.ddReqQueue <- t
|
||||
err := t.WaitToFinish()
|
||||
if err != nil {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: "CreateIndex failed, error = " + err.Error(),
|
||||
}, nil
|
||||
}
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
//TODO
|
||||
func (c *Core) DescribeIndex(in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
|
||||
return nil, nil
|
||||
t := &DescribeIndexReqTask{
|
||||
baseReqTask: baseReqTask{
|
||||
cv: make(chan error),
|
||||
core: c,
|
||||
},
|
||||
Req: in,
|
||||
Rsp: &milvuspb.DescribeIndexResponse{
|
||||
Status: nil,
|
||||
IndexDescriptions: nil,
|
||||
},
|
||||
}
|
||||
c.ddReqQueue <- t
|
||||
err := t.WaitToFinish()
|
||||
if err != nil {
|
||||
return &milvuspb.DescribeIndexResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: "DescribeIndex failed, error = " + err.Error(),
|
||||
},
|
||||
IndexDescriptions: nil,
|
||||
}, nil
|
||||
}
|
||||
t.Rsp.Status = &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
Reason: "",
|
||||
}
|
||||
return t.Rsp, nil
|
||||
}
|
||||
|
||||
//TODO
|
||||
func (c *Core) DescribeSegment(in *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) {
|
||||
return nil, nil
|
||||
t := &DescribeSegmentReqTask{
|
||||
baseReqTask: baseReqTask{
|
||||
cv: make(chan error),
|
||||
core: c,
|
||||
},
|
||||
Req: in,
|
||||
Rsp: &milvuspb.DescribeSegmentResponse{
|
||||
Status: nil,
|
||||
IndexID: 0,
|
||||
},
|
||||
}
|
||||
c.ddReqQueue <- t
|
||||
err := t.WaitToFinish()
|
||||
if err != nil {
|
||||
return &milvuspb.DescribeSegmentResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: "DescribeSegment failed, error = " + err.Error(),
|
||||
},
|
||||
IndexID: 0,
|
||||
}, nil
|
||||
}
|
||||
t.Rsp.Status = &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
Reason: "",
|
||||
}
|
||||
return t.Rsp, nil
|
||||
}
|
||||
|
||||
//TODO
|
||||
func (c *Core) ShowSegments(in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) {
|
||||
return nil, nil
|
||||
t := &ShowSegmentReqTask{
|
||||
baseReqTask: baseReqTask{
|
||||
cv: make(chan error),
|
||||
core: c,
|
||||
},
|
||||
Req: in,
|
||||
Rsp: &milvuspb.ShowSegmentResponse{
|
||||
Status: nil,
|
||||
SegmentIDs: nil,
|
||||
},
|
||||
}
|
||||
c.ddReqQueue <- t
|
||||
err := t.WaitToFinish()
|
||||
if err != nil {
|
||||
return &milvuspb.ShowSegmentResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: "ShowSegments failed, error: " + err.Error(),
|
||||
},
|
||||
SegmentIDs: nil,
|
||||
}, nil
|
||||
}
|
||||
t.Rsp.Status = &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
Reason: "",
|
||||
}
|
||||
return t.Rsp, nil
|
||||
}
|
||||
|
||||
func (c *Core) AllocTimestamp(in *masterpb.TsoRequest) (*masterpb.TsoResponse, error) {
|
||||
|
|
|
@ -9,8 +9,10 @@ import (
|
|||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
)
|
||||
|
||||
|
@ -24,14 +26,16 @@ const (
|
|||
)
|
||||
|
||||
type metaTable struct {
|
||||
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
|
||||
tenantID2Meta map[typeutil.UniqueID]pb.TenantMeta // tenant id to tenant meta
|
||||
proxyID2Meta map[typeutil.UniqueID]pb.ProxyMeta // proxy id to proxy meta
|
||||
collID2Meta map[typeutil.UniqueID]pb.CollectionInfo // collection id to collection meta,
|
||||
collName2ID map[string]typeutil.UniqueID // collection name to collection id
|
||||
partitionID2Meta map[typeutil.UniqueID]pb.PartitionInfo //partition id -> partition meta
|
||||
segID2IndexMeta map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo // segment id -> index id -> segment index meta
|
||||
indexID2Meta map[typeutil.UniqueID]pb.IndexInfo //index id ->index meta
|
||||
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
|
||||
tenantID2Meta map[typeutil.UniqueID]pb.TenantMeta // tenant id to tenant meta
|
||||
proxyID2Meta map[typeutil.UniqueID]pb.ProxyMeta // proxy id to proxy meta
|
||||
collID2Meta map[typeutil.UniqueID]pb.CollectionInfo // collection id to collection meta,
|
||||
collName2ID map[string]typeutil.UniqueID // collection name to collection id
|
||||
partitionID2Meta map[typeutil.UniqueID]pb.PartitionInfo // partition id -> partition meta
|
||||
segID2IndexMeta map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo // segment id -> index id -> segment index meta
|
||||
indexID2Meta map[typeutil.UniqueID]pb.IndexInfo // index id ->index meta
|
||||
segID2CollID map[typeutil.UniqueID]typeutil.UniqueID // segment id -> collection id
|
||||
partitionID2CollID map[typeutil.UniqueID]typeutil.UniqueID // partition id -> collection id
|
||||
|
||||
tenantLock sync.RWMutex
|
||||
proxyLock sync.RWMutex
|
||||
|
@ -61,6 +65,8 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
mt.partitionID2Meta = make(map[typeutil.UniqueID]pb.PartitionInfo)
|
||||
mt.segID2IndexMeta = make(map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo)
|
||||
mt.indexID2Meta = make(map[typeutil.UniqueID]pb.IndexInfo)
|
||||
mt.partitionID2CollID = make(map[typeutil.UniqueID]typeutil.UniqueID)
|
||||
mt.segID2CollID = make(map[typeutil.UniqueID]typeutil.UniqueID)
|
||||
|
||||
_, values, err := mt.client.LoadWithPrefix(TenantMetaPrefix)
|
||||
if err != nil {
|
||||
|
@ -103,6 +109,9 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
}
|
||||
mt.collID2Meta[collectionInfo.ID] = collectionInfo
|
||||
mt.collName2ID[collectionInfo.Schema.Name] = collectionInfo.ID
|
||||
for _, partID := range collectionInfo.PartitionIDs {
|
||||
mt.partitionID2CollID[partID] = collectionInfo.ID
|
||||
}
|
||||
}
|
||||
|
||||
_, values, err = mt.client.LoadWithPrefix(PartitionMetaPrefix)
|
||||
|
@ -115,7 +124,15 @@ func (mt *metaTable) reloadFromKV() error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
collID, ok := mt.partitionID2CollID[partitionInfo.PartitionID]
|
||||
if !ok {
|
||||
log.Printf("partition id %d not belong to any collection", partitionInfo.PartitionID)
|
||||
continue
|
||||
}
|
||||
mt.partitionID2Meta[partitionInfo.PartitionID] = partitionInfo
|
||||
for _, segID := range partitionInfo.SegmentIDs {
|
||||
mt.segID2CollID[segID] = collID
|
||||
}
|
||||
}
|
||||
|
||||
_, values, err = mt.client.LoadWithPrefix(SegmentIndexMetaPrefix)
|
||||
|
@ -169,6 +186,7 @@ func (mt *metaTable) AddCollection(coll *pb.CollectionInfo, part *pb.PartitionIn
|
|||
mt.collID2Meta[coll.ID] = *coll
|
||||
mt.collName2ID[coll.Schema.Name] = coll.ID
|
||||
mt.partitionID2Meta[part.PartitionID] = *part
|
||||
mt.partitionID2CollID[part.PartitionID] = coll.ID
|
||||
|
||||
k1 := path.Join(CollectionMetaPrefix, strconv.FormatInt(coll.ID, 10))
|
||||
v1 := proto.MarshalTextString(coll)
|
||||
|
@ -240,6 +258,17 @@ func (mt *metaTable) HasCollection(collID typeutil.UniqueID) bool {
|
|||
return ok
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetCollectionByID(collectionID typeutil.UniqueID) (pb.CollectionInfo, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
|
||||
col, ok := mt.collID2Meta[collectionID]
|
||||
if !ok {
|
||||
return pb.CollectionInfo{}, errors.Errorf("can't find collection id : %d", collectionID)
|
||||
}
|
||||
return col, nil
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetCollectionByName(collectionName string) (pb.CollectionInfo, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
|
@ -299,6 +328,7 @@ func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string
|
|||
coll.PartitionIDs = append(coll.PartitionIDs, partitionID)
|
||||
mt.partitionID2Meta[partitionID] = partMeta
|
||||
mt.collID2Meta[collID] = coll
|
||||
mt.partitionID2CollID[partitionID] = collID
|
||||
|
||||
k1 := path.Join(CollectionMetaPrefix, strconv.FormatInt(coll.ID, 10))
|
||||
v1 := proto.MarshalTextString(&coll)
|
||||
|
@ -442,6 +472,7 @@ func (mt *metaTable) AddSegment(seg *datapb.SegmentInfo) error {
|
|||
}
|
||||
partMeta.SegmentIDs = append(partMeta.SegmentIDs, seg.SegmentID)
|
||||
mt.partitionID2Meta[seg.PartitionID] = partMeta
|
||||
mt.segID2CollID[seg.SegmentID] = seg.CollectionID
|
||||
err := mt.client.Save(path.Join(PartitionMetaPrefix, strconv.FormatInt(seg.PartitionID, 10)), proto.MarshalTextString(&partMeta))
|
||||
|
||||
if err != nil {
|
||||
|
@ -450,3 +481,202 @@ func (mt *metaTable) AddSegment(seg *datapb.SegmentInfo) error {
|
|||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mt *metaTable) AddIndex(seg *pb.SegmentIndexInfo, idx *pb.IndexInfo) error {
|
||||
mt.ddLock.Lock()
|
||||
defer mt.ddLock.Unlock()
|
||||
|
||||
if seg.IndexID != idx.IndexID {
|
||||
return errors.Errorf("index id in segment is %d, in index info is %d, not equal", seg.IndexID, idx.IndexID)
|
||||
}
|
||||
|
||||
segIdxMap, ok := mt.segID2IndexMeta[seg.SegmentID]
|
||||
if !ok {
|
||||
idxMap := map[typeutil.UniqueID]pb.SegmentIndexInfo{seg.IndexID: *seg}
|
||||
mt.segID2IndexMeta[seg.SegmentID] = &idxMap
|
||||
} else {
|
||||
_, ok := (*segIdxMap)[seg.IndexID]
|
||||
if ok {
|
||||
return errors.Errorf("index id = %d exist", seg.IndexID)
|
||||
}
|
||||
}
|
||||
_, ok = mt.indexID2Meta[idx.IndexID]
|
||||
if ok {
|
||||
return errors.Errorf("index id = %d exist", idx.IndexID)
|
||||
}
|
||||
(*(mt.segID2IndexMeta[seg.SegmentID]))[seg.IndexID] = *seg
|
||||
mt.indexID2Meta[idx.IndexID] = *idx
|
||||
k1 := path.Join(SegmentIndexMetaPrefix, strconv.FormatInt(seg.SegmentID, 10), strconv.FormatInt(seg.IndexID, 10))
|
||||
v1 := proto.MarshalTextString(seg)
|
||||
k2 := path.Join(IndexMetaPrefix, strconv.FormatInt(idx.IndexID, 10))
|
||||
v2 := proto.MarshalTextString(idx)
|
||||
meta := map[string]string{k1: v1, k2: v2}
|
||||
|
||||
err := mt.client.MultiSave(meta)
|
||||
if err != nil {
|
||||
_ = mt.reloadFromKV()
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetSegmentIndexInfoByID(segID typeutil.UniqueID, filedID int64, idxName string) (pb.SegmentIndexInfo, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
|
||||
segIdxMap, ok := mt.segID2IndexMeta[segID]
|
||||
if !ok {
|
||||
return pb.SegmentIndexInfo{}, errors.Errorf("segment id %d not has any index", segID)
|
||||
}
|
||||
if len(*segIdxMap) == 0 {
|
||||
return pb.SegmentIndexInfo{}, errors.Errorf("segment id %d not has any index", segID)
|
||||
}
|
||||
|
||||
if filedID == -1 && idxName == "" { // return any index
|
||||
for _, seg := range *segIdxMap {
|
||||
return seg, nil
|
||||
}
|
||||
} else {
|
||||
for idxID, seg := range *segIdxMap {
|
||||
idxMeta, ok := mt.indexID2Meta[idxID]
|
||||
if ok {
|
||||
if idxMeta.IndexName != idxName {
|
||||
continue
|
||||
}
|
||||
if seg.FieldID != filedID {
|
||||
continue
|
||||
}
|
||||
return seg, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
return pb.SegmentIndexInfo{}, errors.Errorf("can't find index name = %s on segment = %d, with filed id = %d", idxName, segID, filedID)
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetFieldSchema(collName string, fieldName string) (schemapb.FieldSchema, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
|
||||
collID, ok := mt.collName2ID[collName]
|
||||
if !ok {
|
||||
return schemapb.FieldSchema{}, errors.Errorf("collection %s not found", collName)
|
||||
}
|
||||
collMeta, ok := mt.collID2Meta[collID]
|
||||
if !ok {
|
||||
return schemapb.FieldSchema{}, errors.Errorf("collection %s not found", collName)
|
||||
}
|
||||
|
||||
for _, field := range collMeta.Schema.Fields {
|
||||
if field.Name == fieldName {
|
||||
return *field, nil
|
||||
}
|
||||
}
|
||||
return schemapb.FieldSchema{}, errors.Errorf("collection %s doesn't have filed %s", collName, fieldName)
|
||||
}
|
||||
|
||||
//return true/false
|
||||
func (mt *metaTable) IsSegmentIndexed(segID typeutil.UniqueID, fieldSchema *schemapb.FieldSchema, indexParams []*commonpb.KeyValuePair) bool {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
segIdx, ok := mt.segID2IndexMeta[segID]
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
exist := false
|
||||
for idxID, meta := range *segIdx {
|
||||
if meta.FieldID != fieldSchema.FieldID {
|
||||
continue
|
||||
}
|
||||
idxMeta, ok := mt.indexID2Meta[idxID]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if EqualKeyPairArray(indexParams, idxMeta.IndexParams) {
|
||||
exist = true
|
||||
break
|
||||
}
|
||||
}
|
||||
return exist
|
||||
}
|
||||
|
||||
// return segment ids, type params, error
|
||||
func (mt *metaTable) GetNotIndexedSegments(collName string, fieldName string, indexParams []*commonpb.KeyValuePair) ([]typeutil.UniqueID, schemapb.FieldSchema, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
collID, ok := mt.collName2ID[collName]
|
||||
if !ok {
|
||||
return nil, schemapb.FieldSchema{}, errors.Errorf("collection %s not found", collName)
|
||||
}
|
||||
collMeta, ok := mt.collID2Meta[collID]
|
||||
if !ok {
|
||||
return nil, schemapb.FieldSchema{}, errors.Errorf("collection %s not found", collName)
|
||||
}
|
||||
fieldSchema, err := mt.GetFieldSchema(collName, fieldName)
|
||||
if err != nil {
|
||||
return nil, fieldSchema, err
|
||||
}
|
||||
|
||||
rstID := make([]typeutil.UniqueID, 0, 16)
|
||||
for _, partID := range collMeta.PartitionIDs {
|
||||
partMeta, ok := mt.partitionID2Meta[partID]
|
||||
if ok {
|
||||
for _, segID := range partMeta.SegmentIDs {
|
||||
if exist := mt.IsSegmentIndexed(segID, &fieldSchema, indexParams); !exist {
|
||||
rstID = append(rstID, segID)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return rstID, fieldSchema, nil
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetIndexByName(collName string, fieldName string, indexName string) ([]pb.IndexInfo, error) {
|
||||
mt.ddLock.RLock()
|
||||
mt.ddLock.RUnlock()
|
||||
|
||||
collID, ok := mt.collName2ID[collName]
|
||||
if !ok {
|
||||
return nil, errors.Errorf("collection %s not found", collName)
|
||||
}
|
||||
collMeta, ok := mt.collID2Meta[collID]
|
||||
if !ok {
|
||||
return nil, errors.Errorf("collection %s not found", collName)
|
||||
}
|
||||
fileSchema, err := mt.GetFieldSchema(collName, fieldName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rstIndex := make([]*pb.IndexInfo, 0, 16)
|
||||
|
||||
for _, partID := range collMeta.PartitionIDs {
|
||||
partMeta, ok := mt.partitionID2Meta[partID]
|
||||
if ok {
|
||||
for _, segID := range partMeta.SegmentIDs {
|
||||
idxMeta, ok := mt.segID2IndexMeta[segID]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
for idxID, segMeta := range *idxMeta {
|
||||
if segMeta.FieldID != fileSchema.FieldID {
|
||||
continue
|
||||
}
|
||||
idxMeta, ok := mt.indexID2Meta[idxID]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if indexName == "" {
|
||||
rstIndex = append(rstIndex, &idxMeta)
|
||||
} else if idxMeta.IndexName == indexName {
|
||||
rstIndex = append(rstIndex, &idxMeta)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
rst := make([]pb.IndexInfo, 0, len(rstIndex))
|
||||
for i := range rstIndex {
|
||||
rst = append(rst, *rstIndex[i])
|
||||
}
|
||||
return rst, nil
|
||||
}
|
||||
|
|
|
@ -0,0 +1,201 @@
|
|||
package masterservice
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
)
|
||||
|
||||
func TestMetaTable(t *testing.T) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
randVal := rand.Int()
|
||||
etcdAddr := "127.0.0.1:2379"
|
||||
rootPath := fmt.Sprintf("/test/meta/%d", randVal)
|
||||
|
||||
etcdCli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}})
|
||||
assert.Nil(t, err)
|
||||
ekv := etcdkv.NewEtcdKV(etcdCli, rootPath)
|
||||
assert.NotNil(t, ekv)
|
||||
mt, err := NewMetaTable(ekv)
|
||||
assert.Nil(t, err)
|
||||
|
||||
collInfo := &pb.CollectionInfo{
|
||||
ID: 1,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "testColl",
|
||||
Description: "",
|
||||
AutoID: false,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 110,
|
||||
Name: "field110",
|
||||
IsPrimaryKey: false,
|
||||
Description: "",
|
||||
DataType: 0,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-k1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
{
|
||||
Key: "field110-k2",
|
||||
Value: "field110-v2",
|
||||
},
|
||||
},
|
||||
IndexParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-i1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
{
|
||||
Key: "field110-i2",
|
||||
Value: "field110-v2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
CreateTime: 0,
|
||||
PartitionIDs: nil,
|
||||
}
|
||||
partInfo := &pb.PartitionInfo{
|
||||
PartitionName: "testPart",
|
||||
PartitionID: 10,
|
||||
SegmentIDs: nil,
|
||||
}
|
||||
|
||||
t.Run("add collection", func(t *testing.T) {
|
||||
err = mt.AddCollection(collInfo, partInfo)
|
||||
assert.Nil(t, err)
|
||||
|
||||
collMeta, err := mt.GetCollectionByName("testColl")
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, collMeta.PartitionIDs[0], int64(10))
|
||||
assert.Equal(t, len(collMeta.PartitionIDs), 1)
|
||||
})
|
||||
|
||||
t.Run("add segment", func(t *testing.T) {
|
||||
seg := &datapb.SegmentInfo{
|
||||
SegmentID: 100,
|
||||
CollectionID: 1,
|
||||
PartitionID: 10,
|
||||
}
|
||||
assert.Nil(t, mt.AddSegment(seg))
|
||||
assert.NotNil(t, mt.AddSegment(seg))
|
||||
seg.SegmentID = 101
|
||||
seg.CollectionID = 2
|
||||
assert.NotNil(t, mt.AddSegment(seg))
|
||||
seg.CollectionID = 1
|
||||
seg.PartitionID = 11
|
||||
assert.NotNil(t, mt.AddSegment(seg))
|
||||
seg.PartitionID = 10
|
||||
assert.Nil(t, mt.AddSegment(seg))
|
||||
})
|
||||
|
||||
t.Run("add segment index", func(t *testing.T) {
|
||||
seg := pb.SegmentIndexInfo{
|
||||
SegmentID: 100,
|
||||
FieldID: 110,
|
||||
IndexID: 200,
|
||||
BuildID: 201,
|
||||
}
|
||||
idx := pb.IndexInfo{
|
||||
IndexName: "idx200",
|
||||
IndexID: 200,
|
||||
IndexParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-i1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
{
|
||||
Key: "field110-i2",
|
||||
Value: "field110-v2",
|
||||
},
|
||||
},
|
||||
}
|
||||
err := mt.AddIndex(&seg, &idx)
|
||||
assert.Nil(t, err)
|
||||
assert.NotNil(t, mt.AddIndex(&seg, &idx))
|
||||
})
|
||||
|
||||
t.Run("get not indexed segments", func(t *testing.T) {
|
||||
params := []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-i1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
{
|
||||
Key: "field110-i2",
|
||||
Value: "field110-v2",
|
||||
},
|
||||
}
|
||||
|
||||
tparams := []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-k1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
{
|
||||
Key: "field110-k2",
|
||||
Value: "field110-v2",
|
||||
},
|
||||
}
|
||||
|
||||
_, field, err := mt.GetNotIndexedSegments("collTest", "field110", params)
|
||||
assert.NotNil(t, err)
|
||||
seg, field, err := mt.GetNotIndexedSegments("testColl", "field110", params)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(seg), 1)
|
||||
assert.Equal(t, seg[0], int64(101))
|
||||
assert.True(t, EqualKeyPairArray(field.TypeParams, tparams))
|
||||
|
||||
params = []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-i1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
}
|
||||
|
||||
seg, field, err = mt.GetNotIndexedSegments("testColl", "field110", params)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(seg), 2)
|
||||
assert.Equal(t, seg[0], int64(100))
|
||||
assert.Equal(t, seg[1], int64(101))
|
||||
assert.True(t, EqualKeyPairArray(field.TypeParams, tparams))
|
||||
|
||||
})
|
||||
|
||||
t.Run("get index by name", func(t *testing.T) {
|
||||
idx, err := mt.GetIndexByName("testColl", "field110", "idx200")
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(idx), 1)
|
||||
assert.Equal(t, idx[0].IndexID, int64(200))
|
||||
params := []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "field110-i1",
|
||||
Value: "field110-v1",
|
||||
},
|
||||
{
|
||||
Key: "field110-i2",
|
||||
Value: "field110-v2",
|
||||
},
|
||||
}
|
||||
assert.True(t, EqualKeyPairArray(idx[0].IndexParams, params))
|
||||
|
||||
_, err = mt.GetIndexByName("testColl", "field111", "idx200")
|
||||
assert.NotNil(t, err)
|
||||
idx, err = mt.GetIndexByName("testColl", "field110", "idx201")
|
||||
assert.Nil(t, err)
|
||||
assert.Zero(t, len(idx))
|
||||
})
|
||||
|
||||
}
|
|
@ -26,4 +26,5 @@ type ParamTable struct {
|
|||
|
||||
MaxPartitionNum int64
|
||||
DefaultPartitionName string
|
||||
DefaultIndexName string
|
||||
}
|
||||
|
|
|
@ -382,3 +382,189 @@ func (t *ShowPartitionReqTask) Execute() error {
|
|||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type DescribeSegmentReqTask struct {
|
||||
baseReqTask
|
||||
Req *milvuspb.DescribeSegmentRequest
|
||||
Rsp *milvuspb.DescribeSegmentResponse //TODO,return repeated segment id in the future
|
||||
}
|
||||
|
||||
func (t *DescribeSegmentReqTask) Type() commonpb.MsgType {
|
||||
return t.Req.Base.MsgType
|
||||
}
|
||||
|
||||
func (t *DescribeSegmentReqTask) Ts() (typeutil.Timestamp, error) {
|
||||
return t.Req.Base.Timestamp, nil
|
||||
}
|
||||
|
||||
func (t *DescribeSegmentReqTask) Execute() error {
|
||||
coll, err := t.core.MetaTable.GetCollectionByID(t.Req.CollectionID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
exist := false
|
||||
for _, partID := range coll.PartitionIDs {
|
||||
if exist {
|
||||
break
|
||||
}
|
||||
partMeta, err := t.core.MetaTable.GetPartitionByID(partID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, e := range partMeta.SegmentIDs {
|
||||
if e == t.Req.SegmentID {
|
||||
exist = true
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !exist {
|
||||
return errors.Errorf("segment id %d not belong to collection id %d", t.Req.SegmentID, t.Req.CollectionID)
|
||||
}
|
||||
//TODO, get filed_id and index_name from request
|
||||
segIdxInfo, err := t.core.MetaTable.GetSegmentIndexInfoByID(t.Req.SegmentID, -1, "")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t.Rsp.IndexID = segIdxInfo.IndexID
|
||||
return nil
|
||||
}
|
||||
|
||||
type ShowSegmentReqTask struct {
|
||||
baseReqTask
|
||||
Req *milvuspb.ShowSegmentRequest
|
||||
Rsp *milvuspb.ShowSegmentResponse
|
||||
}
|
||||
|
||||
func (t *ShowSegmentReqTask) Type() commonpb.MsgType {
|
||||
return t.Req.Base.MsgType
|
||||
}
|
||||
|
||||
func (t *ShowSegmentReqTask) Ts() (typeutil.Timestamp, error) {
|
||||
return t.Req.Base.Timestamp, nil
|
||||
}
|
||||
|
||||
func (t *ShowSegmentReqTask) Execute() error {
|
||||
coll, err := t.core.MetaTable.GetCollectionByID(t.Req.CollectionID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, partID := range coll.PartitionIDs {
|
||||
partMeta, err := t.core.MetaTable.GetPartitionByID(partID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t.Rsp.SegmentIDs = append(t.Rsp.SegmentIDs, partMeta.SegmentIDs...)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type CreateIndexReqTask struct {
|
||||
baseReqTask
|
||||
Req *milvuspb.CreateIndexRequest
|
||||
}
|
||||
|
||||
func (t *CreateIndexReqTask) Type() commonpb.MsgType {
|
||||
return t.Req.Base.MsgType
|
||||
}
|
||||
|
||||
func (t *CreateIndexReqTask) Ts() (typeutil.Timestamp, error) {
|
||||
return t.Req.Base.Timestamp, nil
|
||||
}
|
||||
|
||||
func (t *CreateIndexReqTask) Execute() error {
|
||||
segIDs, field, err := t.core.MetaTable.GetNotIndexedSegments(t.Req.CollectionName, t.Req.FieldName, t.Req.ExtraParams)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, seg := range segIDs {
|
||||
task := CreateIndexTask{
|
||||
core: t.core,
|
||||
segmentID: seg,
|
||||
indexName: Params.DefaultIndexName, //TODO, get name from request
|
||||
fieldSchema: &field,
|
||||
indexParams: t.Req.ExtraParams,
|
||||
}
|
||||
t.core.indexTaskQueue <- &task
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type DescribeIndexReqTask struct {
|
||||
baseReqTask
|
||||
Req *milvuspb.DescribeIndexRequest
|
||||
Rsp *milvuspb.DescribeIndexResponse
|
||||
}
|
||||
|
||||
func (t *DescribeIndexReqTask) Type() commonpb.MsgType {
|
||||
return t.Req.Base.MsgType
|
||||
}
|
||||
|
||||
func (t *DescribeIndexReqTask) Ts() (typeutil.Timestamp, error) {
|
||||
return t.Req.Base.Timestamp, nil
|
||||
}
|
||||
|
||||
func (t *DescribeIndexReqTask) Execute() error {
|
||||
idx, err := t.core.MetaTable.GetIndexByName(t.Req.CollectionName, t.Req.FieldName, t.Req.IndexName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, i := range idx {
|
||||
desc := &milvuspb.IndexDescription{
|
||||
IndexName: i.IndexName,
|
||||
Params: i.IndexParams,
|
||||
}
|
||||
t.Rsp.IndexDescriptions = append(t.Rsp.IndexDescriptions, desc)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type CreateIndexTask struct {
|
||||
core *Core
|
||||
segmentID typeutil.UniqueID
|
||||
indexName string
|
||||
fieldSchema *schemapb.FieldSchema
|
||||
indexParams []*commonpb.KeyValuePair
|
||||
}
|
||||
|
||||
func (t *CreateIndexTask) BuildIndex() error {
|
||||
if t.core.MetaTable.IsSegmentIndexed(t.segmentID, t.fieldSchema, t.indexParams) {
|
||||
return nil
|
||||
}
|
||||
idxID, err := t.core.idAllocator.AllocOne()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
binlogs, err := t.core.GetBinlogFilePathsFromDataServiceReq(t.segmentID, t.fieldSchema.FieldID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var bldID typeutil.UniqueID
|
||||
|
||||
if len(t.indexParams) == 0 {
|
||||
t.indexParams = make([]*commonpb.KeyValuePair, 0, len(t.fieldSchema.IndexParams))
|
||||
for _, p := range t.fieldSchema.IndexParams {
|
||||
t.indexParams = append(t.indexParams, &commonpb.KeyValuePair{
|
||||
Key: p.Key,
|
||||
Value: p.Value,
|
||||
})
|
||||
}
|
||||
}
|
||||
bldID, err = t.core.BuildIndexReq(binlogs, t.fieldSchema.TypeParams, t.indexParams)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
seg := etcdpb.SegmentIndexInfo{
|
||||
SegmentID: t.segmentID,
|
||||
FieldID: t.fieldSchema.FieldID,
|
||||
IndexID: idxID,
|
||||
BuildID: bldID,
|
||||
}
|
||||
idx := etcdpb.IndexInfo{
|
||||
IndexName: t.indexName,
|
||||
IndexID: idxID,
|
||||
IndexParams: t.indexParams,
|
||||
}
|
||||
err = t.core.MetaTable.AddIndex(&seg, &idx)
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -0,0 +1,24 @@
|
|||
package masterservice
|
||||
|
||||
import "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
|
||||
//return
|
||||
func EqualKeyPairArray(p1 []*commonpb.KeyValuePair, p2 []*commonpb.KeyValuePair) bool {
|
||||
if len(p1) != len(p2) {
|
||||
return false
|
||||
}
|
||||
m1 := make(map[string]string)
|
||||
for _, p := range p1 {
|
||||
m1[p.Key] = p.Value
|
||||
}
|
||||
for _, p := range p2 {
|
||||
val, ok := m1[p.Key]
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
if val != p.Value {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
|
@ -34,9 +34,9 @@ message IndexStatesResponse {
|
|||
}
|
||||
|
||||
message BuildIndexRequest {
|
||||
repeated string data_paths = 2;
|
||||
repeated common.KeyValuePair type_params = 3;
|
||||
repeated common.KeyValuePair index_params = 4;
|
||||
repeated string data_paths = 1;
|
||||
repeated common.KeyValuePair type_params = 2;
|
||||
repeated common.KeyValuePair index_params = 3;
|
||||
}
|
||||
|
||||
message BuildIndexResponse {
|
||||
|
@ -56,14 +56,19 @@ message BuildIndexNotification {
|
|||
repeated string index_file_paths = 3;
|
||||
}
|
||||
|
||||
message IndexFilePathRequest {
|
||||
int64 indexID = 1;
|
||||
message IndexFilePathsRequest {
|
||||
repeated int64 indexIDs = 1;
|
||||
}
|
||||
|
||||
message IndexFilePathInfo {
|
||||
common.Status status = 1;
|
||||
int64 indexID = 2;
|
||||
repeated string index_file_paths = 3;
|
||||
}
|
||||
|
||||
message IndexFilePathsResponse {
|
||||
common.Status status = 1;
|
||||
int64 indexID = 2;
|
||||
repeated string index_file_paths = 3;
|
||||
repeated IndexFilePathInfo file_paths = 2;
|
||||
}
|
||||
|
||||
message IndexMeta {
|
||||
|
@ -88,7 +93,7 @@ service IndexService {
|
|||
rpc RegisterNode(RegisterNodeRequest) returns (RegisterNodeResponse) {}
|
||||
rpc BuildIndex(BuildIndexRequest) returns (BuildIndexResponse){}
|
||||
rpc GetIndexStates(IndexStatesRequest) returns (IndexStatesResponse) {}
|
||||
rpc GetIndexFilePaths(IndexFilePathRequest) returns (IndexFilePathsResponse){}
|
||||
rpc GetIndexFilePaths(IndexFilePathsRequest) returns (IndexFilePathsResponse){}
|
||||
rpc NotifyBuildIndex(BuildIndexNotification) returns (common.Status) {}
|
||||
}
|
||||
|
||||
|
|
|
@ -262,9 +262,9 @@ func (m *IndexStatesResponse) GetStates() []*IndexInfo {
|
|||
}
|
||||
|
||||
type BuildIndexRequest struct {
|
||||
DataPaths []string `protobuf:"bytes,2,rep,name=data_paths,json=dataPaths,proto3" json:"data_paths,omitempty"`
|
||||
TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"`
|
||||
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,4,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||
DataPaths []string `protobuf:"bytes,1,rep,name=data_paths,json=dataPaths,proto3" json:"data_paths,omitempty"`
|
||||
TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,2,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"`
|
||||
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
|
@ -465,46 +465,46 @@ func (m *BuildIndexNotification) GetIndexFilePaths() []string {
|
|||
return nil
|
||||
}
|
||||
|
||||
type IndexFilePathRequest struct {
|
||||
IndexID int64 `protobuf:"varint,1,opt,name=indexID,proto3" json:"indexID,omitempty"`
|
||||
type IndexFilePathsRequest struct {
|
||||
IndexIDs []int64 `protobuf:"varint,1,rep,packed,name=indexIDs,proto3" json:"indexIDs,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *IndexFilePathRequest) Reset() { *m = IndexFilePathRequest{} }
|
||||
func (m *IndexFilePathRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*IndexFilePathRequest) ProtoMessage() {}
|
||||
func (*IndexFilePathRequest) Descriptor() ([]byte, []int) {
|
||||
func (m *IndexFilePathsRequest) Reset() { *m = IndexFilePathsRequest{} }
|
||||
func (m *IndexFilePathsRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*IndexFilePathsRequest) ProtoMessage() {}
|
||||
func (*IndexFilePathsRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_a5d2036b4df73e0a, []int{9}
|
||||
}
|
||||
|
||||
func (m *IndexFilePathRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_IndexFilePathRequest.Unmarshal(m, b)
|
||||
func (m *IndexFilePathsRequest) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_IndexFilePathsRequest.Unmarshal(m, b)
|
||||
}
|
||||
func (m *IndexFilePathRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_IndexFilePathRequest.Marshal(b, m, deterministic)
|
||||
func (m *IndexFilePathsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_IndexFilePathsRequest.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *IndexFilePathRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_IndexFilePathRequest.Merge(m, src)
|
||||
func (m *IndexFilePathsRequest) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_IndexFilePathsRequest.Merge(m, src)
|
||||
}
|
||||
func (m *IndexFilePathRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_IndexFilePathRequest.Size(m)
|
||||
func (m *IndexFilePathsRequest) XXX_Size() int {
|
||||
return xxx_messageInfo_IndexFilePathsRequest.Size(m)
|
||||
}
|
||||
func (m *IndexFilePathRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_IndexFilePathRequest.DiscardUnknown(m)
|
||||
func (m *IndexFilePathsRequest) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_IndexFilePathsRequest.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_IndexFilePathRequest proto.InternalMessageInfo
|
||||
var xxx_messageInfo_IndexFilePathsRequest proto.InternalMessageInfo
|
||||
|
||||
func (m *IndexFilePathRequest) GetIndexID() int64 {
|
||||
func (m *IndexFilePathsRequest) GetIndexIDs() []int64 {
|
||||
if m != nil {
|
||||
return m.IndexID
|
||||
return m.IndexIDs
|
||||
}
|
||||
return 0
|
||||
return nil
|
||||
}
|
||||
|
||||
type IndexFilePathsResponse struct {
|
||||
type IndexFilePathInfo struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
IndexID int64 `protobuf:"varint,2,opt,name=indexID,proto3" json:"indexID,omitempty"`
|
||||
IndexFilePaths []string `protobuf:"bytes,3,rep,name=index_file_paths,json=indexFilePaths,proto3" json:"index_file_paths,omitempty"`
|
||||
|
@ -513,11 +513,65 @@ type IndexFilePathsResponse struct {
|
|||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *IndexFilePathInfo) Reset() { *m = IndexFilePathInfo{} }
|
||||
func (m *IndexFilePathInfo) String() string { return proto.CompactTextString(m) }
|
||||
func (*IndexFilePathInfo) ProtoMessage() {}
|
||||
func (*IndexFilePathInfo) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_a5d2036b4df73e0a, []int{10}
|
||||
}
|
||||
|
||||
func (m *IndexFilePathInfo) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_IndexFilePathInfo.Unmarshal(m, b)
|
||||
}
|
||||
func (m *IndexFilePathInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_IndexFilePathInfo.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *IndexFilePathInfo) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_IndexFilePathInfo.Merge(m, src)
|
||||
}
|
||||
func (m *IndexFilePathInfo) XXX_Size() int {
|
||||
return xxx_messageInfo_IndexFilePathInfo.Size(m)
|
||||
}
|
||||
func (m *IndexFilePathInfo) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_IndexFilePathInfo.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_IndexFilePathInfo proto.InternalMessageInfo
|
||||
|
||||
func (m *IndexFilePathInfo) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *IndexFilePathInfo) GetIndexID() int64 {
|
||||
if m != nil {
|
||||
return m.IndexID
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *IndexFilePathInfo) GetIndexFilePaths() []string {
|
||||
if m != nil {
|
||||
return m.IndexFilePaths
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type IndexFilePathsResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
FilePaths []*IndexFilePathInfo `protobuf:"bytes,2,rep,name=file_paths,json=filePaths,proto3" json:"file_paths,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *IndexFilePathsResponse) Reset() { *m = IndexFilePathsResponse{} }
|
||||
func (m *IndexFilePathsResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*IndexFilePathsResponse) ProtoMessage() {}
|
||||
func (*IndexFilePathsResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_a5d2036b4df73e0a, []int{10}
|
||||
return fileDescriptor_a5d2036b4df73e0a, []int{11}
|
||||
}
|
||||
|
||||
func (m *IndexFilePathsResponse) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -545,16 +599,9 @@ func (m *IndexFilePathsResponse) GetStatus() *commonpb.Status {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *IndexFilePathsResponse) GetIndexID() int64 {
|
||||
func (m *IndexFilePathsResponse) GetFilePaths() []*IndexFilePathInfo {
|
||||
if m != nil {
|
||||
return m.IndexID
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *IndexFilePathsResponse) GetIndexFilePaths() []string {
|
||||
if m != nil {
|
||||
return m.IndexFilePaths
|
||||
return m.FilePaths
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -576,7 +623,7 @@ func (m *IndexMeta) Reset() { *m = IndexMeta{} }
|
|||
func (m *IndexMeta) String() string { return proto.CompactTextString(m) }
|
||||
func (*IndexMeta) ProtoMessage() {}
|
||||
func (*IndexMeta) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_a5d2036b4df73e0a, []int{11}
|
||||
return fileDescriptor_a5d2036b4df73e0a, []int{12}
|
||||
}
|
||||
|
||||
func (m *IndexMeta) XXX_Unmarshal(b []byte) error {
|
||||
|
@ -656,7 +703,8 @@ func init() {
|
|||
proto.RegisterType((*BuildIndexResponse)(nil), "milvus.proto.index.BuildIndexResponse")
|
||||
proto.RegisterType((*BuildIndexCmd)(nil), "milvus.proto.index.BuildIndexCmd")
|
||||
proto.RegisterType((*BuildIndexNotification)(nil), "milvus.proto.index.BuildIndexNotification")
|
||||
proto.RegisterType((*IndexFilePathRequest)(nil), "milvus.proto.index.IndexFilePathRequest")
|
||||
proto.RegisterType((*IndexFilePathsRequest)(nil), "milvus.proto.index.IndexFilePathsRequest")
|
||||
proto.RegisterType((*IndexFilePathInfo)(nil), "milvus.proto.index.IndexFilePathInfo")
|
||||
proto.RegisterType((*IndexFilePathsResponse)(nil), "milvus.proto.index.IndexFilePathsResponse")
|
||||
proto.RegisterType((*IndexMeta)(nil), "milvus.proto.index.IndexMeta")
|
||||
}
|
||||
|
@ -664,55 +712,57 @@ func init() {
|
|||
func init() { proto.RegisterFile("index_service.proto", fileDescriptor_a5d2036b4df73e0a) }
|
||||
|
||||
var fileDescriptor_a5d2036b4df73e0a = []byte{
|
||||
// 757 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x5d, 0x4f, 0xdb, 0x4a,
|
||||
0x10, 0xc5, 0x38, 0x04, 0x65, 0x12, 0x22, 0xd8, 0x20, 0x14, 0xe5, 0x5e, 0x74, 0xc1, 0x57, 0x17,
|
||||
0x22, 0xa4, 0xeb, 0xa0, 0x20, 0xda, 0xc7, 0x8a, 0x80, 0x5a, 0x45, 0x15, 0x08, 0xb9, 0x55, 0x1f,
|
||||
0x5a, 0x55, 0x91, 0x63, 0x0f, 0x64, 0x55, 0x7f, 0x04, 0xef, 0x1a, 0x15, 0x5e, 0xaa, 0xaa, 0x3f,
|
||||
0xa0, 0xea, 0x6f, 0xe9, 0x6b, 0x7f, 0x5c, 0xe5, 0xdd, 0x75, 0x12, 0x83, 0x49, 0x40, 0xd0, 0x37,
|
||||
0xef, 0xee, 0x99, 0x33, 0xb3, 0xe7, 0xcc, 0xac, 0xa1, 0x46, 0x03, 0x17, 0x3f, 0xf7, 0x18, 0x46,
|
||||
0x97, 0xd4, 0x41, 0x73, 0x18, 0x85, 0x3c, 0x24, 0xc4, 0xa7, 0xde, 0x65, 0xcc, 0xe4, 0xca, 0x14,
|
||||
0x88, 0x46, 0xc5, 0x09, 0x7d, 0x3f, 0x0c, 0xe4, 0x5e, 0xa3, 0x4a, 0x03, 0x8e, 0x51, 0x60, 0x7b,
|
||||
0x72, 0x6d, 0x7c, 0x81, 0x9a, 0x85, 0xe7, 0x94, 0x71, 0x8c, 0x4e, 0x42, 0x17, 0x2d, 0xbc, 0x88,
|
||||
0x91, 0x71, 0xb2, 0x0b, 0x85, 0xbe, 0xcd, 0xb0, 0xae, 0x6d, 0x68, 0xcd, 0x72, 0xfb, 0x6f, 0x33,
|
||||
0xc3, 0xab, 0x08, 0x8f, 0xd9, 0x79, 0xc7, 0x66, 0x68, 0x09, 0x24, 0x79, 0x06, 0x8b, 0xb6, 0xeb,
|
||||
0x46, 0xc8, 0x58, 0x7d, 0x7e, 0x4a, 0xd0, 0x81, 0xc4, 0x58, 0x29, 0xd8, 0xf8, 0xae, 0xc1, 0x6a,
|
||||
0xb6, 0x02, 0x36, 0x0c, 0x03, 0x86, 0x64, 0x0f, 0x8a, 0x8c, 0xdb, 0x3c, 0x66, 0xaa, 0x88, 0xbf,
|
||||
0x72, 0xf9, 0xde, 0x08, 0x88, 0xa5, 0xa0, 0xa4, 0x03, 0x65, 0x1a, 0x50, 0xde, 0x1b, 0xda, 0x91,
|
||||
0xed, 0xa7, 0x95, 0x6c, 0x9a, 0x37, 0x64, 0x51, 0x0a, 0x74, 0x03, 0xca, 0x4f, 0x05, 0xd0, 0x02,
|
||||
0x3a, 0xfa, 0x36, 0x4c, 0x20, 0xdd, 0x44, 0xb9, 0x84, 0x1a, 0x59, 0xaa, 0x48, 0x1d, 0x16, 0x85,
|
||||
0x9e, 0xdd, 0xa3, 0xba, 0xb6, 0xa1, 0x37, 0x75, 0x2b, 0x5d, 0x1a, 0x1c, 0x4a, 0x02, 0xdf, 0x0d,
|
||||
0xce, 0x42, 0xb2, 0x0f, 0x0b, 0x49, 0x29, 0x52, 0xb9, 0x6a, 0xfb, 0x9f, 0xdc, 0xa2, 0xc7, 0xf4,
|
||||
0x96, 0x44, 0x4f, 0xb2, 0x27, 0x35, 0x8f, 0xd9, 0xc9, 0x1a, 0x14, 0x2d, 0xb4, 0x59, 0x18, 0xd4,
|
||||
0xf5, 0x0d, 0xad, 0x59, 0xb2, 0xd4, 0xca, 0xf8, 0xaa, 0x41, 0x2d, 0x53, 0xe6, 0x63, 0x64, 0xdb,
|
||||
0x97, 0x41, 0x98, 0x28, 0xa6, 0x37, 0xcb, 0xed, 0x75, 0xf3, 0x76, 0x23, 0x99, 0xa3, 0x4b, 0x5a,
|
||||
0x0a, 0x6c, 0xfc, 0xd2, 0x60, 0xa5, 0x13, 0x53, 0xcf, 0x15, 0x47, 0xa9, 0x52, 0xeb, 0x00, 0xae,
|
||||
0xcd, 0xed, 0xde, 0xd0, 0xe6, 0x03, 0x49, 0x58, 0xb2, 0x4a, 0xc9, 0xce, 0x69, 0xb2, 0x91, 0x58,
|
||||
0xc4, 0xaf, 0x86, 0x98, 0x5a, 0xa4, 0x8b, 0x84, 0x9b, 0xb9, 0x55, 0xbe, 0xc6, 0xab, 0x77, 0xb6,
|
||||
0x17, 0xe3, 0xa9, 0x4d, 0x23, 0x0b, 0x92, 0x28, 0x69, 0x11, 0x39, 0x82, 0x8a, 0x6c, 0x7f, 0x45,
|
||||
0x52, 0xb8, 0x2f, 0x49, 0x59, 0x84, 0x29, 0xa3, 0x1d, 0x20, 0x93, 0xd5, 0x3f, 0x46, 0xc0, 0x3b,
|
||||
0xfd, 0x33, 0xfa, 0xb0, 0x34, 0x4e, 0x72, 0xe8, 0xbb, 0xd9, 0x46, 0xca, 0x58, 0xfd, 0x1c, 0xf4,
|
||||
0x08, 0x2f, 0x54, 0xd3, 0xfe, 0x97, 0x67, 0xc1, 0x2d, 0xb1, 0xad, 0x24, 0xc2, 0xf8, 0xa1, 0xc1,
|
||||
0xda, 0xf8, 0xe8, 0x24, 0xe4, 0xf4, 0x8c, 0x3a, 0x36, 0xa7, 0x61, 0xf0, 0xc4, 0xb7, 0x21, 0x4d,
|
||||
0x58, 0x96, 0xc2, 0x9f, 0x51, 0x0f, 0x95, 0xc3, 0xba, 0x70, 0xb8, 0x2a, 0xf6, 0x5f, 0x52, 0x0f,
|
||||
0x85, 0xcd, 0xc6, 0x2e, 0xac, 0x76, 0x27, 0x77, 0x72, 0xe7, 0x28, 0xa3, 0x54, 0x72, 0x8b, 0x4c,
|
||||
0x08, 0xfb, 0x43, 0x9e, 0x3c, 0xe0, 0x16, 0x3f, 0xe7, 0xd5, 0x70, 0x1f, 0x23, 0xb7, 0x9f, 0x7e,
|
||||
0xb8, 0xd7, 0x01, 0x30, 0xb8, 0x88, 0xb1, 0xc7, 0xa9, 0x8f, 0x62, 0xc0, 0x75, 0xab, 0x24, 0x76,
|
||||
0xde, 0x52, 0x1f, 0xc9, 0xbf, 0xb0, 0xc4, 0x9c, 0x01, 0xba, 0xb1, 0xa7, 0x10, 0x05, 0x81, 0xa8,
|
||||
0xa4, 0x9b, 0x02, 0x64, 0x42, 0xad, 0x9f, 0x78, 0xdf, 0x73, 0x42, 0x7f, 0xe8, 0x21, 0x57, 0xd0,
|
||||
0x05, 0x01, 0x5d, 0x11, 0x47, 0x87, 0xea, 0x44, 0xe0, 0x55, 0x97, 0x15, 0x1f, 0xda, 0x65, 0xb9,
|
||||
0xaa, 0x2d, 0xe6, 0xa9, 0xd6, 0xfe, 0x56, 0x80, 0x8a, 0x94, 0x41, 0xfe, 0x9d, 0x88, 0x03, 0x95,
|
||||
0xc9, 0x37, 0x9e, 0x6c, 0xe7, 0xa5, 0xcd, 0xf9, 0x0f, 0x35, 0x9a, 0xb3, 0x81, 0xb2, 0x45, 0x8c,
|
||||
0x39, 0xf2, 0x11, 0x60, 0x5c, 0x39, 0xb9, 0xdf, 0xcd, 0x1a, 0x5b, 0xb3, 0x60, 0x23, 0x7a, 0x07,
|
||||
0xaa, 0xaf, 0x90, 0x4f, 0x3c, 0xb9, 0x64, 0xeb, 0xce, 0x57, 0x32, 0xf3, 0xeb, 0x68, 0x6c, 0xcf,
|
||||
0xc4, 0x8d, 0x92, 0x7c, 0x82, 0x95, 0x34, 0xc9, 0x48, 0x4e, 0xd2, 0xbc, 0x33, 0xfe, 0xc6, 0x70,
|
||||
0x35, 0x76, 0x66, 0x22, 0x59, 0x46, 0xb0, 0x65, 0xf1, 0x56, 0x5c, 0x4d, 0xc8, 0xb6, 0x33, 0x5d,
|
||||
0x8f, 0xc9, 0xb7, 0xa5, 0x31, 0x6d, 0x0a, 0x8d, 0xb9, 0xf6, 0x07, 0x35, 0x3a, 0xc2, 0xf1, 0x93,
|
||||
0x8c, 0x39, 0x9b, 0xd3, 0xb3, 0x1c, 0xfa, 0xee, 0x0c, 0xf2, 0xce, 0xc1, 0xfb, 0x17, 0xe7, 0x94,
|
||||
0x0f, 0xe2, 0x7e, 0x72, 0xd2, 0xba, 0xa6, 0x9e, 0x47, 0xaf, 0x39, 0x3a, 0x83, 0x96, 0x8c, 0xfa,
|
||||
0xdf, 0xa5, 0x8c, 0x47, 0xb4, 0x1f, 0x73, 0x74, 0x5b, 0xe9, 0x0f, 0xbf, 0x25, 0xa8, 0x5a, 0x22,
|
||||
0xdb, 0xb0, 0xdf, 0x2f, 0x8a, 0xe5, 0xde, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4d, 0x32, 0xc8,
|
||||
0x07, 0x4a, 0x09, 0x00, 0x00,
|
||||
// 793 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x5d, 0x4f, 0xe3, 0x46,
|
||||
0x14, 0xc5, 0x18, 0x42, 0x73, 0x13, 0x22, 0x32, 0x69, 0x51, 0x94, 0x16, 0x15, 0x5c, 0x01, 0x29,
|
||||
0x52, 0x9d, 0x2a, 0x88, 0xf6, 0xb1, 0x22, 0xa0, 0x56, 0x51, 0x05, 0x42, 0xd3, 0xaa, 0x0f, 0xad,
|
||||
0xaa, 0xc8, 0xb1, 0x6f, 0xc8, 0x48, 0xfe, 0x08, 0x9e, 0x31, 0x5a, 0x78, 0x59, 0xad, 0xb4, 0x8f,
|
||||
0x2b, 0xad, 0x56, 0xfb, 0x53, 0xf6, 0x75, 0x7f, 0xdc, 0xca, 0xe3, 0xb1, 0x63, 0x43, 0x48, 0x58,
|
||||
0x81, 0xb4, 0x6f, 0x99, 0xeb, 0x73, 0x3f, 0xe6, 0x9c, 0x33, 0x33, 0x81, 0x06, 0xf3, 0x1d, 0x7c,
|
||||
0x31, 0xe0, 0x18, 0x5e, 0x33, 0x1b, 0xcd, 0x49, 0x18, 0x88, 0x80, 0x10, 0x8f, 0xb9, 0xd7, 0x11,
|
||||
0x4f, 0x56, 0xa6, 0x44, 0xb4, 0xaa, 0x76, 0xe0, 0x79, 0x81, 0x9f, 0xc4, 0x5a, 0x35, 0xe6, 0x0b,
|
||||
0x0c, 0x7d, 0xcb, 0x4d, 0xd6, 0xc6, 0x4b, 0x68, 0x50, 0xbc, 0x64, 0x5c, 0x60, 0x78, 0x1e, 0x38,
|
||||
0x48, 0xf1, 0x2a, 0x42, 0x2e, 0xc8, 0xcf, 0xb0, 0x32, 0xb4, 0x38, 0x36, 0xb5, 0x6d, 0xad, 0x5d,
|
||||
0xe9, 0x7e, 0x67, 0x16, 0xea, 0xaa, 0x82, 0x67, 0xfc, 0xb2, 0x67, 0x71, 0xa4, 0x12, 0x49, 0x7e,
|
||||
0x81, 0x35, 0xcb, 0x71, 0x42, 0xe4, 0xbc, 0xb9, 0x3c, 0x27, 0xe9, 0x38, 0xc1, 0xd0, 0x14, 0x6c,
|
||||
0xbc, 0xd5, 0xe0, 0xeb, 0xe2, 0x04, 0x7c, 0x12, 0xf8, 0x1c, 0xc9, 0x21, 0x94, 0xb8, 0xb0, 0x44,
|
||||
0xc4, 0xd5, 0x10, 0xdf, 0xce, 0xac, 0xf7, 0x97, 0x84, 0x50, 0x05, 0x25, 0x3d, 0xa8, 0x30, 0x9f,
|
||||
0x89, 0xc1, 0xc4, 0x0a, 0x2d, 0x2f, 0x9d, 0x64, 0xc7, 0xbc, 0x43, 0x8b, 0x62, 0xa0, 0xef, 0x33,
|
||||
0x71, 0x21, 0x81, 0x14, 0x58, 0xf6, 0xdb, 0x30, 0x81, 0xf4, 0x63, 0xe6, 0xe2, 0xd2, 0xc8, 0x53,
|
||||
0x46, 0x9a, 0xb0, 0x26, 0xf9, 0xec, 0x9f, 0x36, 0xb5, 0x6d, 0xbd, 0xad, 0xd3, 0x74, 0x69, 0x08,
|
||||
0x28, 0x4b, 0x7c, 0xdf, 0x1f, 0x05, 0xe4, 0x08, 0x56, 0xe3, 0x51, 0x12, 0xe6, 0x6a, 0xdd, 0xef,
|
||||
0x67, 0x0e, 0x3d, 0x2d, 0x4f, 0x13, 0x74, 0xbe, 0x7a, 0x3c, 0xf3, 0xb4, 0x3a, 0xd9, 0x84, 0x12,
|
||||
0x45, 0x8b, 0x07, 0x7e, 0x53, 0xdf, 0xd6, 0xda, 0x65, 0xaa, 0x56, 0xc6, 0x2b, 0x0d, 0x1a, 0x85,
|
||||
0x31, 0x9f, 0x42, 0xdb, 0x51, 0x92, 0x84, 0x31, 0x63, 0x7a, 0xbb, 0xd2, 0xdd, 0x32, 0xef, 0x1b,
|
||||
0xc9, 0xcc, 0x36, 0x49, 0x15, 0xd8, 0xf8, 0xa8, 0x41, 0xbd, 0x17, 0x31, 0xd7, 0x91, 0x9f, 0x52,
|
||||
0xa6, 0xb6, 0x00, 0x1c, 0x4b, 0x58, 0x83, 0x89, 0x25, 0xc6, 0x5c, 0x92, 0x55, 0xa6, 0xe5, 0x38,
|
||||
0x72, 0x11, 0x07, 0x62, 0x89, 0xc4, 0xcd, 0x04, 0xa7, 0x12, 0xe9, 0xf7, 0x25, 0x52, 0x53, 0xfe,
|
||||
0x89, 0x37, 0xff, 0x58, 0x6e, 0x84, 0x17, 0x16, 0x0b, 0x29, 0xc4, 0x59, 0x89, 0x44, 0xe4, 0x14,
|
||||
0xaa, 0x89, 0xfd, 0x55, 0x11, 0xfd, 0xb1, 0x45, 0x2a, 0x32, 0x4d, 0x09, 0x6d, 0x03, 0xc9, 0x4f,
|
||||
0xff, 0x14, 0x02, 0x1f, 0xd4, 0xcf, 0x18, 0xc2, 0xfa, 0xb4, 0xc9, 0x89, 0xe7, 0x14, 0x8d, 0x54,
|
||||
0x90, 0xfa, 0x57, 0xd0, 0x43, 0xbc, 0x52, 0xa6, 0xdd, 0x9d, 0x25, 0xc1, 0x3d, 0xb2, 0x69, 0x9c,
|
||||
0x61, 0xbc, 0xd3, 0x60, 0x73, 0xfa, 0xe9, 0x3c, 0x10, 0x6c, 0xc4, 0x6c, 0x4b, 0xb0, 0xc0, 0x7f,
|
||||
0xe6, 0xdd, 0x90, 0x36, 0x6c, 0x24, 0xc4, 0x8f, 0x98, 0x8b, 0x4a, 0x61, 0x5d, 0x2a, 0x5c, 0x93,
|
||||
0xf1, 0xdf, 0x99, 0x8b, 0x52, 0x66, 0xe3, 0x10, 0xbe, 0xe9, 0x17, 0x22, 0xa9, 0x3d, 0x5a, 0xf0,
|
||||
0x95, 0xaa, 0xc6, 0xd5, 0x49, 0xca, 0xd6, 0xc6, 0x1b, 0x0d, 0xea, 0x85, 0x2c, 0x79, 0xa6, 0xbe,
|
||||
0xd8, 0x1e, 0xde, 0x6b, 0xb0, 0x79, 0x77, 0x13, 0x4f, 0x71, 0xc9, 0x29, 0x40, 0xae, 0x67, 0xe2,
|
||||
0xfc, 0xdd, 0x07, 0x8f, 0x5a, 0x9e, 0x03, 0x5a, 0x1e, 0x65, 0x53, 0x7d, 0x58, 0x56, 0x17, 0xce,
|
||||
0x19, 0x0a, 0xeb, 0xf9, 0x2f, 0x9c, 0x2d, 0x00, 0xf4, 0xaf, 0x22, 0x1c, 0x08, 0xe6, 0xa1, 0xbc,
|
||||
0x74, 0x74, 0x5a, 0x96, 0x91, 0xbf, 0x99, 0x87, 0xe4, 0x07, 0x58, 0xe7, 0xf6, 0x18, 0x9d, 0xc8,
|
||||
0x55, 0x88, 0x15, 0x89, 0xa8, 0xa6, 0x41, 0x09, 0x32, 0xa1, 0x31, 0x8c, 0xfd, 0x38, 0xb0, 0x03,
|
||||
0x6f, 0xe2, 0xa2, 0x50, 0xd0, 0x55, 0x09, 0xad, 0xcb, 0x4f, 0x27, 0xea, 0x8b, 0xc4, 0x2b, 0xe7,
|
||||
0x97, 0x3e, 0xd7, 0xf9, 0x33, 0xb5, 0x5c, 0x9b, 0xa5, 0x65, 0xf7, 0xf5, 0x0a, 0x54, 0x13, 0x1a,
|
||||
0x92, 0x17, 0x93, 0xd8, 0x50, 0xcd, 0xbf, 0x3b, 0x64, 0x7f, 0x56, 0xdb, 0x19, 0x6f, 0x63, 0xab,
|
||||
0xbd, 0x18, 0x98, 0x98, 0xc4, 0x58, 0x22, 0xff, 0x03, 0x4c, 0x27, 0x27, 0x8f, 0xdb, 0x59, 0x6b,
|
||||
0x6f, 0x11, 0x2c, 0x2b, 0x6f, 0x43, 0xed, 0x0f, 0x14, 0xb9, 0x67, 0x80, 0xec, 0x3d, 0x68, 0xa7,
|
||||
0xc2, 0x73, 0xd6, 0xda, 0x5f, 0x88, 0xcb, 0x9a, 0xb8, 0x50, 0x4f, 0x9b, 0x64, 0x74, 0x92, 0x1f,
|
||||
0x17, 0xda, 0x36, 0x6b, 0x75, 0xf0, 0x18, 0x68, 0x8e, 0xb1, 0x0d, 0x79, 0x81, 0xdd, 0xe4, 0x78,
|
||||
0x3b, 0x98, 0x4f, 0x48, 0xfe, 0xc2, 0x6b, 0xcd, 0x3b, 0x88, 0xc6, 0x52, 0xf7, 0x3f, 0x75, 0x76,
|
||||
0xa4, 0xe4, 0xe7, 0x05, 0x75, 0x76, 0xe6, 0x77, 0x39, 0xf1, 0x9c, 0x05, 0xc5, 0x7b, 0xc7, 0xff,
|
||||
0xfe, 0x76, 0xc9, 0xc4, 0x38, 0x1a, 0xc6, 0x5f, 0x3a, 0xb7, 0xcc, 0x75, 0xd9, 0xad, 0x40, 0x7b,
|
||||
0xdc, 0x49, 0xb2, 0x7e, 0x72, 0x18, 0x17, 0x21, 0x1b, 0x46, 0x02, 0x9d, 0x4e, 0xfa, 0x2f, 0xa4,
|
||||
0x23, 0x4b, 0x75, 0x64, 0xb7, 0xc9, 0x70, 0x58, 0x92, 0xcb, 0xc3, 0x4f, 0x01, 0x00, 0x00, 0xff,
|
||||
0xff, 0x10, 0x1a, 0x0a, 0xe9, 0xdf, 0x09, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
|
@ -736,7 +786,7 @@ type IndexServiceClient interface {
|
|||
RegisterNode(ctx context.Context, in *RegisterNodeRequest, opts ...grpc.CallOption) (*RegisterNodeResponse, error)
|
||||
BuildIndex(ctx context.Context, in *BuildIndexRequest, opts ...grpc.CallOption) (*BuildIndexResponse, error)
|
||||
GetIndexStates(ctx context.Context, in *IndexStatesRequest, opts ...grpc.CallOption) (*IndexStatesResponse, error)
|
||||
GetIndexFilePaths(ctx context.Context, in *IndexFilePathRequest, opts ...grpc.CallOption) (*IndexFilePathsResponse, error)
|
||||
GetIndexFilePaths(ctx context.Context, in *IndexFilePathsRequest, opts ...grpc.CallOption) (*IndexFilePathsResponse, error)
|
||||
NotifyBuildIndex(ctx context.Context, in *BuildIndexNotification, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
}
|
||||
|
||||
|
@ -775,7 +825,7 @@ func (c *indexServiceClient) GetIndexStates(ctx context.Context, in *IndexStates
|
|||
return out, nil
|
||||
}
|
||||
|
||||
func (c *indexServiceClient) GetIndexFilePaths(ctx context.Context, in *IndexFilePathRequest, opts ...grpc.CallOption) (*IndexFilePathsResponse, error) {
|
||||
func (c *indexServiceClient) GetIndexFilePaths(ctx context.Context, in *IndexFilePathsRequest, opts ...grpc.CallOption) (*IndexFilePathsResponse, error) {
|
||||
out := new(IndexFilePathsResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.index.IndexService/GetIndexFilePaths", in, out, opts...)
|
||||
if err != nil {
|
||||
|
@ -804,7 +854,7 @@ type IndexServiceServer interface {
|
|||
RegisterNode(context.Context, *RegisterNodeRequest) (*RegisterNodeResponse, error)
|
||||
BuildIndex(context.Context, *BuildIndexRequest) (*BuildIndexResponse, error)
|
||||
GetIndexStates(context.Context, *IndexStatesRequest) (*IndexStatesResponse, error)
|
||||
GetIndexFilePaths(context.Context, *IndexFilePathRequest) (*IndexFilePathsResponse, error)
|
||||
GetIndexFilePaths(context.Context, *IndexFilePathsRequest) (*IndexFilePathsResponse, error)
|
||||
NotifyBuildIndex(context.Context, *BuildIndexNotification) (*commonpb.Status, error)
|
||||
}
|
||||
|
||||
|
@ -821,7 +871,7 @@ func (*UnimplementedIndexServiceServer) BuildIndex(ctx context.Context, req *Bui
|
|||
func (*UnimplementedIndexServiceServer) GetIndexStates(ctx context.Context, req *IndexStatesRequest) (*IndexStatesResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetIndexStates not implemented")
|
||||
}
|
||||
func (*UnimplementedIndexServiceServer) GetIndexFilePaths(ctx context.Context, req *IndexFilePathRequest) (*IndexFilePathsResponse, error) {
|
||||
func (*UnimplementedIndexServiceServer) GetIndexFilePaths(ctx context.Context, req *IndexFilePathsRequest) (*IndexFilePathsResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method GetIndexFilePaths not implemented")
|
||||
}
|
||||
func (*UnimplementedIndexServiceServer) NotifyBuildIndex(ctx context.Context, req *BuildIndexNotification) (*commonpb.Status, error) {
|
||||
|
@ -887,7 +937,7 @@ func _IndexService_GetIndexStates_Handler(srv interface{}, ctx context.Context,
|
|||
}
|
||||
|
||||
func _IndexService_GetIndexFilePaths_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(IndexFilePathRequest)
|
||||
in := new(IndexFilePathsRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -899,7 +949,7 @@ func _IndexService_GetIndexFilePaths_Handler(srv interface{}, ctx context.Contex
|
|||
FullMethod: "/milvus.proto.index.IndexService/GetIndexFilePaths",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(IndexServiceServer).GetIndexFilePaths(ctx, req.(*IndexFilePathRequest))
|
||||
return srv.(IndexServiceServer).GetIndexFilePaths(ctx, req.(*IndexFilePathsRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
|
|
@ -13,15 +13,15 @@ type ProxyService struct {
|
|||
|
||||
}
|
||||
|
||||
func (s ProxyService) Init() {
|
||||
func (s ProxyService) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (s ProxyService) Start() {
|
||||
func (s ProxyService) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (s ProxyService) Stop() {
|
||||
func (s ProxyService) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -80,7 +80,7 @@ func newServiceTimeNode(ctx context.Context, replica collectionReplica) *service
|
|||
|
||||
timeTimeMsgStream := pulsarms.NewPulsarMsgStream(ctx, Params.SearchReceiveBufSize)
|
||||
timeTimeMsgStream.SetPulsarClient(Params.PulsarAddress)
|
||||
timeTimeMsgStream.CreatePulsarProducers([]string{Params.QueryNodeTimeTickChannelName})
|
||||
timeTimeMsgStream.CreatePulsarProducers([]string{Params.QueryTimeTickChannelName})
|
||||
|
||||
return &serviceTimeNode{
|
||||
baseNode: baseNode,
|
||||
|
|
|
@ -4,6 +4,7 @@ import (
|
|||
"log"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
)
|
||||
|
@ -15,12 +16,12 @@ type ParamTable struct {
|
|||
ETCDAddress string
|
||||
MetaRootPath string
|
||||
|
||||
QueryNodeIP string
|
||||
QueryNodePort int64
|
||||
QueryNodeID UniqueID
|
||||
QueryNodeNum int
|
||||
QueryNodeTimeTickChannelName string
|
||||
QueryNodeTimeTickReceiveBufSize int64
|
||||
QueryNodeIP string
|
||||
QueryNodePort int64
|
||||
QueryNodeID UniqueID
|
||||
QueryNodeNum int
|
||||
QueryTimeTickChannelName string
|
||||
QueryTimeTickReceiveBufSize int64
|
||||
|
||||
FlowGraphMaxQueueLength int32
|
||||
FlowGraphMaxParallelism int32
|
||||
|
@ -90,6 +91,23 @@ func (p *ParamTable) Init() {
|
|||
}
|
||||
}
|
||||
|
||||
queryNodeAddress := os.Getenv("QUERY_NODE_ADDRESS")
|
||||
if queryNodeAddress == "" {
|
||||
p.QueryNodeIP = "localhost"
|
||||
p.QueryNodePort = 20010
|
||||
} else {
|
||||
ipAndPort := strings.Split(queryNodeAddress, ":")
|
||||
if len(ipAndPort) != 2 {
|
||||
panic("illegal query node address")
|
||||
}
|
||||
p.QueryNodeIP = ipAndPort[0]
|
||||
port, err := strconv.ParseInt(ipAndPort[1], 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.QueryNodePort = port
|
||||
}
|
||||
|
||||
err = p.LoadYaml("advanced/common.yaml")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
@ -99,12 +117,10 @@ func (p *ParamTable) Init() {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
p.initQueryNodeIP()
|
||||
p.initQueryNodePort()
|
||||
p.initQueryNodeID()
|
||||
p.initQueryNodeNum()
|
||||
p.initQueryNodeTimeTickChannelName()
|
||||
p.initQueryNodeTimeTickReceiveBufSize()
|
||||
p.initQueryTimeTickChannelName()
|
||||
p.initQueryTimeTickReceiveBufSize()
|
||||
|
||||
p.initMinioEndPoint()
|
||||
p.initMinioAccessKeyID()
|
||||
|
@ -149,25 +165,6 @@ func (p *ParamTable) Init() {
|
|||
}
|
||||
|
||||
// ---------------------------------------------------------- query node
|
||||
func (p *ParamTable) initQueryNodeIP() {
|
||||
ip, err := p.Load("queryNode.ip")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.QueryNodeIP = ip
|
||||
}
|
||||
|
||||
func (p *ParamTable) initQueryNodePort() {
|
||||
port, err := p.Load("queryNode.port")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
p.QueryNodePort, err = strconv.ParseInt(port, 10, 64)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (p *ParamTable) initQueryNodeID() {
|
||||
queryNodeID, err := p.Load("_queryNodeID")
|
||||
if err != nil {
|
||||
|
@ -184,16 +181,16 @@ func (p *ParamTable) initQueryNodeNum() {
|
|||
p.QueryNodeNum = len(p.QueryNodeIDList())
|
||||
}
|
||||
|
||||
func (p *ParamTable) initQueryNodeTimeTickChannelName() {
|
||||
ch, err := p.Load("msgChannel.chanNamePrefix.queryNodeTimeTick")
|
||||
func (p *ParamTable) initQueryTimeTickChannelName() {
|
||||
ch, err := p.Load("msgChannel.chanNamePrefix.queryTimeTick")
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
p.QueryNodeTimeTickChannelName = ch
|
||||
p.QueryTimeTickChannelName = ch
|
||||
}
|
||||
|
||||
func (p *ParamTable) initQueryNodeTimeTickReceiveBufSize() {
|
||||
p.QueryNodeTimeTickReceiveBufSize = p.ParseInt64("queryNode.msgStream.timeTick.recvBufSize")
|
||||
func (p *ParamTable) initQueryTimeTickReceiveBufSize() {
|
||||
p.QueryTimeTickReceiveBufSize = p.ParseInt64("queryNode.msgStream.timeTick.recvBufSize")
|
||||
}
|
||||
|
||||
// ---------------------------------------------------------- minio
|
||||
|
|
|
@ -16,16 +16,6 @@ func TestParamTable_PulsarAddress(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestParamTable_QueryNode(t *testing.T) {
|
||||
t.Run("Test ip", func(t *testing.T) {
|
||||
ip := Params.QueryNodeIP
|
||||
assert.Equal(t, ip, "localhost")
|
||||
})
|
||||
|
||||
t.Run("Test port", func(t *testing.T) {
|
||||
port := Params.QueryNodePort
|
||||
assert.Equal(t, port, int64(20010))
|
||||
})
|
||||
|
||||
t.Run("Test id", func(t *testing.T) {
|
||||
id := Params.QueryNodeID
|
||||
assert.Contains(t, Params.QueryNodeIDList(), id)
|
||||
|
@ -37,12 +27,12 @@ func TestParamTable_QueryNode(t *testing.T) {
|
|||
})
|
||||
|
||||
t.Run("Test time tick channel", func(t *testing.T) {
|
||||
ch := Params.QueryNodeTimeTickChannelName
|
||||
assert.Equal(t, ch, "queryNodeTimeTick")
|
||||
ch := Params.QueryTimeTickChannelName
|
||||
assert.Equal(t, ch, "queryTimeTick")
|
||||
})
|
||||
|
||||
t.Run("Test time tick ReceiveBufSize", func(t *testing.T) {
|
||||
size := Params.QueryNodeTimeTickReceiveBufSize
|
||||
size := Params.QueryTimeTickReceiveBufSize
|
||||
assert.Equal(t, size, int64(64))
|
||||
})
|
||||
}
|
||||
|
|
|
@ -16,10 +16,13 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
queryserviceimpl "github.com/zilliztech/milvus-distributed/internal/queryservice"
|
||||
"io"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
|
||||
queryserviceimpl "github.com/zilliztech/milvus-distributed/internal/queryservice"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/uber/jaeger-client-go/config"
|
||||
|
||||
|
@ -31,9 +34,7 @@ import (
|
|||
)
|
||||
|
||||
type Node interface {
|
||||
Init()
|
||||
Start()
|
||||
Stop()
|
||||
typeutil.Service
|
||||
|
||||
GetComponentStates() (*internalpb2.ComponentStates, error)
|
||||
GetTimeTickChannel() (string, error)
|
||||
|
@ -123,7 +124,7 @@ func Init() {
|
|||
Params.Init()
|
||||
}
|
||||
|
||||
func (node *QueryNode) Init() {
|
||||
func (node *QueryNode) Init() error {
|
||||
registerReq := queryPb.RegisterNodeRequest{
|
||||
Address: &commonpb.Address{
|
||||
Ip: Params.QueryNodeIP,
|
||||
|
@ -141,9 +142,10 @@ func (node *QueryNode) Init() {
|
|||
// TODO: use response.initParams
|
||||
|
||||
Params.Init()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (node *QueryNode) Start() {
|
||||
func (node *QueryNode) Start() error {
|
||||
// todo add connectMaster logic
|
||||
// init services and manager
|
||||
node.dataSyncService = newDataSyncService(node.queryNodeLoopCtx, node.replica)
|
||||
|
@ -162,9 +164,10 @@ func (node *QueryNode) Start() {
|
|||
|
||||
node.stateCode.Store(internalpb2.StateCode_HEALTHY)
|
||||
<-node.queryNodeLoopCtx.Done()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (node *QueryNode) Stop() {
|
||||
func (node *QueryNode) Stop() error {
|
||||
node.stateCode.Store(internalpb2.StateCode_ABNORMAL)
|
||||
node.queryNodeLoopCancel()
|
||||
|
||||
|
@ -187,6 +190,7 @@ func (node *QueryNode) Stop() {
|
|||
if node.closer != nil {
|
||||
node.closer.Close()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (node *QueryNode) GetComponentStates() (*internalpb2.ComponentStates, error) {
|
||||
|
@ -206,7 +210,7 @@ func (node *QueryNode) GetComponentStates() (*internalpb2.ComponentStates, error
|
|||
}
|
||||
|
||||
func (node *QueryNode) GetTimeTickChannel() (string, error) {
|
||||
return Params.QueryNodeTimeTickChannelName, nil
|
||||
return Params.QueryTimeTickChannelName, nil
|
||||
}
|
||||
|
||||
func (node *QueryNode) GetStatisticsChannel() (string, error) {
|
||||
|
|
|
@ -227,15 +227,15 @@ func (s *segmentManager) loadSegmentFieldsData(segmentID UniqueID, targetFields
|
|||
}
|
||||
|
||||
func (s *segmentManager) getIndexPaths(indexID UniqueID) ([]string, error) {
|
||||
indexFilePathRequest := &indexpb.IndexFilePathRequest{
|
||||
IndexID: indexID,
|
||||
indexFilePathRequest := &indexpb.IndexFilePathsRequest{
|
||||
IndexIDs: []UniqueID{indexID},
|
||||
}
|
||||
pathResponse, err := s.indexBuilderClient.GetIndexFilePaths(context.TODO(), indexFilePathRequest)
|
||||
if err != nil || pathResponse.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return pathResponse.IndexFilePaths, nil
|
||||
return pathResponse.FilePaths[0].IndexFilePaths, nil
|
||||
}
|
||||
|
||||
func (s *segmentManager) getIndexParam() (indexParam, error) {
|
||||
|
|
|
@ -9,15 +9,15 @@ type QueryService struct {
|
|||
}
|
||||
|
||||
//serverBase interface
|
||||
func (qs *QueryService) Init() {
|
||||
func (qs *QueryService) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (qs *QueryService) Start() {
|
||||
func (qs *QueryService) Start() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (qs *QueryService) Stop() {
|
||||
func (qs *QueryService) Stop() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
|
|
|
@ -5,9 +5,9 @@ import (
|
|||
)
|
||||
|
||||
type Service interface {
|
||||
Init()
|
||||
Start()
|
||||
Stop()
|
||||
Init() error
|
||||
Start() error
|
||||
Stop() error
|
||||
}
|
||||
|
||||
type Component interface {
|
||||
|
|
Loading…
Reference in New Issue