Add create index

Signed-off-by: neza2017 <yefu.chen@zilliz.com>
pull/4973/head^2
neza2017 2021-01-21 10:01:29 +08:00 committed by yefu.chen
parent 7f044fff82
commit 9828a52999
34 changed files with 1217 additions and 235 deletions

View File

@ -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"

View File

@ -52,8 +52,6 @@ proxyNode:
port: 19530
queryNode:
ip: localhost
port: 20010
gracefulTime: 5000 #ms
indexBuilder:

View File

@ -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)

View File

@ -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")
}

View File

@ -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()

View File

@ -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)
}

View File

@ -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) {

View File

@ -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")
}

View File

@ -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")
}

View File

@ -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")
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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")
}

View File

@ -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])
}

View File

@ -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")
}

View File

@ -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)
}

View File

@ -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 {

View File

@ -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

View File

@ -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) {

View File

@ -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
}

View File

@ -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))
})
}

View File

@ -26,4 +26,5 @@ type ParamTable struct {
MaxPartitionNum int64
DefaultPartitionName string
DefaultIndexName string
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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) {}
}

View File

@ -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)
}

View File

@ -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")
}

View File

@ -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,

View File

@ -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

View File

@ -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))
})
}

View File

@ -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) {

View File

@ -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) {

View File

@ -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")
}

View File

@ -5,9 +5,9 @@ import (
)
type Service interface {
Init()
Start()
Stop()
Init() error
Start() error
Stop() error
}
type Component interface {