Add workflow of drop index in index service

Signed-off-by: dragondriver <jiquan.long@zilliz.com>
pull/4973/head^2
dragondriver 2021-02-23 11:57:18 +08:00 committed by yefu.chen
parent bbfcbbdd68
commit b506754364
12 changed files with 258 additions and 64 deletions

View File

@ -61,6 +61,11 @@ func (c *Client) BuildIndex(req *indexpb.BuildIndexCmd) (*commonpb.Status, error
return c.grpcClient.BuildIndex(ctx, req)
}
func (c *Client) DropIndex(req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
ctx := context.TODO()
return c.grpcClient.DropIndex(ctx, req)
}
func NewClient(nodeAddress string) (*Client, error) {
return &Client{

View File

@ -137,6 +137,10 @@ func (s *Server) BuildIndex(ctx context.Context, req *indexpb.BuildIndexCmd) (*c
return s.impl.BuildIndex(req)
}
func (s *Server) DropIndex(ctx context.Context, request *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return s.impl.DropIndex(request)
}
func (s *Server) GetComponentStates(ctx context.Context, empty *commonpb.Empty) (*internalpb2.ComponentStates, error) {
return s.impl.GetComponentStates()
}

View File

@ -32,10 +32,6 @@ type Server struct {
loopWg sync.WaitGroup
}
func (s *Server) DropIndex(ctx context.Context, request *indexpb.DropIndexRequest) (*commonpb.Status, error) {
panic("implement me")
}
func (s *Server) Run() error {
if err := s.init(); err != nil {
@ -103,6 +99,11 @@ func (s *Server) GetIndexStates(ctx context.Context, req *indexpb.IndexStatesReq
return s.impl.GetIndexStates(req)
}
func (s *Server) DropIndex(ctx context.Context, request *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return s.impl.DropIndex(request)
}
func (s *Server) GetIndexFilePaths(ctx context.Context, req *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error) {
return s.impl.GetIndexFilePaths(req)

View File

@ -189,6 +189,14 @@ func (i *NodeImpl) BuildIndex(request *indexpb.BuildIndexCmd) (*commonpb.Status,
return ret, nil
}
func (i *NodeImpl) DropIndex(request *indexpb.DropIndexRequest) (*commonpb.Status, error) {
i.sched.IndexBuildQueue.tryToRemoveUselessIndexBuildTask(request.IndexID)
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
Reason: "",
}, nil
}
// AddStartCallback adds a callback in the startServer phase.
func (i *NodeImpl) AddStartCallback(callbacks ...func()) {
i.startCallbacks = append(i.startCallbacks, callbacks...)

View File

@ -23,6 +23,7 @@ type TaskQueue interface {
AddActiveTask(t task)
PopActiveTask(tID UniqueID) task
Enqueue(t task) error
tryToRemoveUselessIndexBuildTask(indexID UniqueID)
}
type BaseTaskQueue struct {
@ -116,6 +117,23 @@ func (queue *BaseTaskQueue) PopActiveTask(tID UniqueID) task {
return nil
}
func (queue *BaseTaskQueue) tryToRemoveUselessIndexBuildTask(indexID UniqueID) {
queue.utLock.Lock()
defer queue.utLock.Unlock()
var next *list.Element
for e := queue.unissuedTasks.Front(); e != nil; e = next {
next = e.Next()
indexBuildTask, ok := e.Value.(*IndexBuildTask)
if !ok {
continue
}
if indexBuildTask.cmd.Req.IndexID == indexID {
queue.unissuedTasks.Remove(e)
}
}
}
func (queue *BaseTaskQueue) Enqueue(t task) error {
err := t.OnEnqueue()
if err != nil {

View File

@ -254,6 +254,35 @@ func (i *ServiceImpl) GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.
return ret, nil
}
func (i *ServiceImpl) DropIndex(req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
i.sched.IndexAddQueue.tryToRemoveUselessIndexAddTask(req.IndexID)
err := i.metaTable.MarkIndexAsDeleted(req.IndexID)
if err != nil {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: err.Error(),
}, nil
}
defer func() {
go func() {
allNodeClients := i.nodeClients.PeekAllClients()
for _, client := range allNodeClients {
client.DropIndex(req)
}
}()
go func() {
i.metaTable.removeIndexFile(req.IndexID)
i.metaTable.removeMeta(req.IndexID)
}()
}()
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
}, nil
}
func (i *ServiceImpl) GetIndexFilePaths(req *indexpb.IndexFilePathsRequest) (*indexpb.IndexFilePathsResponse, error) {
var indexPaths []*indexpb.IndexFilePathInfo = nil

View File

@ -13,6 +13,7 @@ package indexservice
import (
"fmt"
"log"
"strconv"
"sync"
@ -25,7 +26,7 @@ import (
type metaTable struct {
client kv.TxnBase // client of a reliable kv service, i.e. etcd client
indexBuildID2Meta map[UniqueID]indexpb.IndexMeta // index id to index meta
indexBuildID2Meta map[UniqueID]indexpb.IndexMeta // index build id to index meta
lock sync.RWMutex
}
@ -86,6 +87,26 @@ func (mt *metaTable) AddIndex(indexBuildID UniqueID, req *indexpb.BuildIndexRequ
return mt.saveIndexMeta(meta)
}
func (mt *metaTable) MarkIndexAsDeleted(indexID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()
exist := false
for indexBuildID, meta := range mt.indexBuildID2Meta {
if meta.Req.IndexID == indexID {
meta.State = commonpb.IndexState_DELETED
mt.indexBuildID2Meta[indexBuildID] = meta
exist = true
}
}
if !exist {
return errors.New("index not exists")
}
return nil
}
func (mt *metaTable) NotifyBuildIndex(nty *indexpb.BuildIndexNotification) error {
mt.lock.Lock()
defer mt.lock.Unlock()
@ -94,6 +115,9 @@ func (mt *metaTable) NotifyBuildIndex(nty *indexpb.BuildIndexNotification) error
if !ok {
return errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexBuildID, 10))
}
if meta.State == commonpb.IndexState_DELETED {
return errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexBuildID, 10))
}
if nty.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
meta.State = commonpb.IndexState_FAILED
@ -117,6 +141,9 @@ func (mt *metaTable) GetIndexState(indexBuildID UniqueID) (*indexpb.IndexInfo, e
if !ok {
return ret, errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexBuildID, 10))
}
if meta.State == commonpb.IndexState_DELETED {
return ret, errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexBuildID, 10))
}
ret.IndexID = meta.Req.IndexID
ret.IndexName = meta.Req.IndexName
ret.Reason = meta.FailReason
@ -134,10 +161,43 @@ func (mt *metaTable) GetIndexFilePathInfo(indexBuildID UniqueID) (*indexpb.Index
if !ok {
return nil, errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexBuildID, 10))
}
if meta.State == commonpb.IndexState_DELETED {
return nil, errors.Errorf("index not exists with ID = " + strconv.FormatInt(indexBuildID, 10))
}
ret.IndexFilePaths = meta.IndexFilePaths
return ret, nil
}
func (mt *metaTable) removeIndexFile(indexID UniqueID) {
mt.lock.Lock()
defer mt.lock.Unlock()
for _, meta := range mt.indexBuildID2Meta {
if meta.Req.IndexID == indexID {
err := mt.client.MultiRemove(meta.IndexFilePaths)
if err != nil {
log.Println("remove index file err: ", err)
}
}
}
}
func (mt *metaTable) removeMeta(indexID UniqueID) {
mt.lock.Lock()
defer mt.lock.Unlock()
indexBuildIDToRemove := make([]UniqueID, 0)
for indexBuildID, meta := range mt.indexBuildID2Meta {
if meta.Req.IndexID == indexID {
indexBuildIDToRemove = append(indexBuildIDToRemove, indexBuildID)
}
}
for _, indexBuildID := range indexBuildIDToRemove {
delete(mt.indexBuildID2Meta, indexBuildID)
}
}
func (mt *metaTable) DeleteIndex(indexBuildID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()

View File

@ -131,3 +131,15 @@ func (pq *PriorityQueue) PeekClient() (UniqueID, typeutil.IndexNodeInterface) {
}
return item.(*PQItem).key, item.(*PQItem).value
}
func (pq *PriorityQueue) PeekAllClients() []typeutil.IndexNodeInterface {
pq.lock.RLock()
defer pq.lock.RUnlock()
var ret []typeutil.IndexNodeInterface
for _, item := range pq.items {
ret = append(ret, item.value)
}
return ret
}

View File

@ -23,6 +23,7 @@ type TaskQueue interface {
AddActiveTask(t task)
PopActiveTask(tID UniqueID) task
Enqueue(t task) error
tryToRemoveUselessIndexAddTask(indexID UniqueID)
}
type BaseTaskQueue struct {
@ -138,6 +139,24 @@ func (queue *IndexAddTaskQueue) Enqueue(t task) error {
return queue.BaseTaskQueue.Enqueue(t)
}
// Note: tryToRemoveUselessIndexAddTask must be called by DropIndex
func (queue *IndexAddTaskQueue) tryToRemoveUselessIndexAddTask(indexID UniqueID) {
queue.lock.Lock()
defer queue.lock.Unlock()
var next *list.Element
for e := queue.unissuedTasks.Front(); e != nil; e = next {
next = e.Next()
indexAddTask, ok := e.Value.(*IndexAddTask)
if !ok {
continue
}
if indexAddTask.req.IndexID == indexID {
queue.unissuedTasks.Remove(e)
}
}
}
func NewIndexAddTaskQueue(sched *TaskScheduler) *IndexAddTaskQueue {
return &IndexAddTaskQueue{
BaseTaskQueue: BaseTaskQueue{

View File

@ -119,6 +119,7 @@ service IndexNode {
* @return Status
*/
rpc BuildIndex(BuildIndexCmd) returns (common.Status){}
rpc DropIndex(DropIndexRequest) returns (common.Status) {}
rpc GetComponentStates(common.Empty) returns (internal.ComponentStates) {}
rpc GetTimeTickChannel(common.Empty) returns(milvus.StringResponse) {}

View File

@ -777,65 +777,65 @@ func init() {
func init() { proto.RegisterFile("index_service.proto", fileDescriptor_a5d2036b4df73e0a) }
var fileDescriptor_a5d2036b4df73e0a = []byte{
// 921 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0x5f, 0x6f, 0xdb, 0x54,
0x14, 0xaf, 0xeb, 0x36, 0x53, 0x4e, 0xb2, 0xaa, 0xbd, 0x1d, 0x55, 0x14, 0x98, 0xd6, 0x9a, 0xfd,
0x09, 0x13, 0x24, 0x28, 0xd3, 0x40, 0x42, 0x42, 0x68, 0x6d, 0xa0, 0x8a, 0xd0, 0xaa, 0xea, 0x6e,
0x02, 0x31, 0x09, 0x55, 0x37, 0xf6, 0x49, 0x7b, 0x85, 0x7d, 0xed, 0xf9, 0xde, 0x4c, 0x74, 0x2f,
0x88, 0x4f, 0xc0, 0x03, 0x12, 0x5f, 0x84, 0x07, 0xbe, 0x11, 0x0f, 0xf0, 0xcc, 0x3b, 0xf2, 0xbd,
0xb6, 0x63, 0x27, 0x5e, 0x92, 0xaa, 0x20, 0x5e, 0x78, 0xcb, 0x39, 0x3e, 0x7f, 0x7f, 0xe7, 0x77,
0xce, 0x0d, 0xec, 0x72, 0xe1, 0xe1, 0xf7, 0x67, 0x12, 0xe3, 0x57, 0xdc, 0xc5, 0x6e, 0x14, 0x87,
0x2a, 0x24, 0x24, 0xe0, 0xfe, 0xab, 0x89, 0x34, 0x52, 0x57, 0x5b, 0xb4, 0x9b, 0x6e, 0x18, 0x04,
0xa1, 0x30, 0xba, 0xf6, 0x16, 0x17, 0x0a, 0x63, 0xc1, 0xfc, 0x54, 0x6e, 0x16, 0x3d, 0x9c, 0x1f,
0x60, 0x97, 0xe2, 0x39, 0x97, 0x0a, 0xe3, 0x93, 0xd0, 0x43, 0x8a, 0x2f, 0x27, 0x28, 0x15, 0xf9,
0x10, 0x36, 0x46, 0x4c, 0x62, 0xcb, 0xda, 0xb7, 0x3a, 0x8d, 0xfe, 0x3b, 0xdd, 0x52, 0x96, 0x34,
0xfc, 0x53, 0x79, 0x7e, 0xc8, 0x24, 0x52, 0x6d, 0x49, 0x3e, 0x82, 0x1b, 0xcc, 0xf3, 0x62, 0x94,
0xb2, 0xb5, 0xbe, 0xc0, 0xe9, 0x89, 0xb1, 0xa1, 0x99, 0xb1, 0xf3, 0x93, 0x05, 0xb7, 0xca, 0x15,
0xc8, 0x28, 0x14, 0x12, 0xc9, 0x23, 0xa8, 0x49, 0xc5, 0xd4, 0x44, 0xa6, 0x45, 0xbc, 0x5d, 0x19,
0xef, 0x99, 0x36, 0xa1, 0xa9, 0x29, 0x39, 0x84, 0x06, 0x17, 0x5c, 0x9d, 0x45, 0x2c, 0x66, 0x41,
0x56, 0xc9, 0x41, 0x77, 0x06, 0xa4, 0x14, 0x8f, 0xa1, 0xe0, 0xea, 0x54, 0x1b, 0x52, 0xe0, 0xf9,
0x6f, 0xe7, 0x13, 0x20, 0xc3, 0x04, 0xc7, 0x24, 0x34, 0xca, 0x0c, 0x91, 0xbb, 0x70, 0x53, 0xa3,
0x7b, 0x38, 0xe1, 0xbe, 0x37, 0x1c, 0x24, 0x55, 0xd9, 0x1d, 0x9b, 0x96, 0x95, 0xce, 0x6f, 0x16,
0xd4, 0xb5, 0xf3, 0x50, 0x8c, 0x43, 0xf2, 0x18, 0x36, 0x93, 0xba, 0x0c, 0x8c, 0x5b, 0xfd, 0x3b,
0x95, 0x1d, 0x4c, 0x73, 0x51, 0x63, 0x4d, 0x1c, 0x68, 0x16, 0xa3, 0xea, 0x2e, 0x6c, 0x5a, 0xd2,
0x91, 0x16, 0xdc, 0xd0, 0xf2, 0x70, 0xd0, 0xb2, 0xf5, 0xe7, 0x4c, 0x24, 0xb7, 0x01, 0x0c, 0x51,
0x04, 0x0b, 0xb0, 0xb5, 0xb1, 0x6f, 0x75, 0xea, 0xb4, 0xae, 0x35, 0x27, 0x2c, 0x40, 0xb2, 0x07,
0x35, 0x8a, 0x4c, 0x86, 0xa2, 0xb5, 0xa9, 0x3f, 0xa5, 0x92, 0xf3, 0xa3, 0x05, 0xbb, 0xa5, 0xb6,
0xaf, 0x33, 0x86, 0xc7, 0xc6, 0x09, 0x93, 0x09, 0xd8, 0x9d, 0x46, 0xff, 0x76, 0x77, 0x9e, 0xa6,
0xdd, 0x1c, 0x27, 0x9a, 0x1a, 0x3b, 0x7f, 0x59, 0xb0, 0x63, 0x1a, 0x4c, 0x3e, 0x65, 0xc8, 0x97,
0x1b, 0xb2, 0x66, 0x1b, 0x2a, 0x20, 0xb1, 0x3e, 0x87, 0x84, 0xc7, 0x14, 0x3b, 0x8b, 0x98, 0xba,
0x90, 0x2d, 0x7b, 0xdf, 0x4e, 0x1c, 0x13, 0xcd, 0x69, 0xa2, 0x48, 0xb8, 0xa2, 0x2e, 0x23, 0xcc,
0xb8, 0xb2, 0xa1, 0x2b, 0x3d, 0xa8, 0x6c, 0xef, 0x4b, 0xbc, 0xfc, 0x8a, 0xf9, 0x13, 0x3c, 0x65,
0x3c, 0xa6, 0x90, 0x78, 0x19, 0xae, 0x90, 0x41, 0x3a, 0xaa, 0x2c, 0xc8, 0xe6, 0xaa, 0x41, 0x1a,
0xda, 0x2d, 0x65, 0x5c, 0x00, 0xa4, 0xd8, 0xf6, 0x75, 0x90, 0x5f, 0x81, 0x3b, 0x8e, 0x0f, 0x37,
0xa7, 0xe9, 0x8e, 0x02, 0x6f, 0xce, 0xc9, 0xaa, 0x20, 0xdc, 0xc7, 0x60, 0xc7, 0xf8, 0x32, 0xdd,
0xa8, 0x7b, 0x55, 0xf3, 0x9c, 0x9b, 0x1c, 0x4d, 0x3c, 0x9c, 0x5f, 0x2d, 0xd8, 0x9b, 0x7e, 0x3a,
0x09, 0x15, 0x1f, 0x73, 0x97, 0x29, 0x1e, 0x8a, 0x7f, 0xad, 0x43, 0xd2, 0x81, 0x6d, 0x33, 0x96,
0x31, 0xf7, 0xb1, 0x34, 0xff, 0x2d, 0xad, 0xff, 0x82, 0xfb, 0x68, 0x48, 0xb0, 0x07, 0x35, 0x11,
0x7a, 0x38, 0x1c, 0xe8, 0x4d, 0xb1, 0x69, 0x2a, 0x39, 0x9f, 0xc2, 0x5b, 0xc3, 0x92, 0xe5, 0xd5,
0xee, 0xc0, 0x2f, 0x16, 0xec, 0x94, 0xfc, 0xf5, 0x3d, 0xf8, 0xef, 0xfb, 0x75, 0x7e, 0xb6, 0x60,
0x6f, 0xb6, 0xb1, 0xeb, 0xf0, 0x6d, 0x00, 0x50, 0xc8, 0x69, 0xb6, 0xfd, 0xde, 0x1b, 0xb7, 0xbd,
0x88, 0x06, 0xad, 0x8f, 0xf3, 0xaa, 0xfe, 0xc8, 0xce, 0xe6, 0x53, 0x54, 0x6c, 0x25, 0x3a, 0xe6,
0xa7, 0x75, 0xfd, 0x4a, 0xa7, 0xf5, 0x0e, 0x34, 0xc6, 0x8c, 0xfb, 0x67, 0xb1, 0x39, 0x81, 0xb6,
0x3e, 0x26, 0x90, 0xa8, 0xcc, 0x19, 0xcc, 0x68, 0xbe, 0x71, 0x55, 0x9a, 0x57, 0x8e, 0x60, 0xb3,
0x72, 0x04, 0xef, 0xc3, 0xf6, 0x20, 0x0e, 0xa3, 0xd2, 0x8d, 0x2b, 0x1c, 0x31, 0xab, 0x74, 0xc4,
0xfa, 0xbf, 0xd7, 0xa0, 0x69, 0xfa, 0x30, 0xef, 0x3e, 0x71, 0xa1, 0x59, 0x7c, 0x2f, 0xc9, 0x83,
0xaa, 0x22, 0x2b, 0xde, 0xf4, 0x76, 0x67, 0xb9, 0xa1, 0x61, 0x82, 0xb3, 0x46, 0xbe, 0x05, 0x98,
0xf6, 0x49, 0x56, 0xc3, 0xa1, 0x7d, 0x7f, 0x99, 0x59, 0x1e, 0xde, 0x85, 0xad, 0x63, 0x54, 0x85,
0xe7, 0x86, 0xdc, 0x7f, 0x23, 0x67, 0x4a, 0xcf, 0x70, 0xfb, 0xc1, 0x52, 0xbb, 0x3c, 0x89, 0x0f,
0x3b, 0x59, 0x92, 0xe9, 0xbe, 0xbf, 0xb7, 0x94, 0x9b, 0x79, 0xaa, 0x87, 0xab, 0x98, 0x16, 0x10,
0xdb, 0xd6, 0xb7, 0xed, 0xb2, 0x80, 0xdb, 0xc3, 0xc5, 0x80, 0x14, 0x6f, 0x61, 0x7b, 0xd1, 0xb6,
0x39, 0x6b, 0xe4, 0x14, 0xea, 0x39, 0x69, 0xc8, 0xdd, 0xaa, 0xb8, 0xb3, 0x9c, 0x5a, 0x16, 0xf1,
0x05, 0x90, 0x63, 0x54, 0x47, 0x61, 0x10, 0x85, 0x02, 0x85, 0x4a, 0xe7, 0xd0, 0xae, 0x74, 0xfa,
0x3c, 0x88, 0xd4, 0xe5, 0xfc, 0x7c, 0xd3, 0xff, 0x51, 0x33, 0x31, 0x9c, 0x35, 0xf2, 0xb5, 0x8e,
0xfd, 0x9c, 0x07, 0xf8, 0x9c, 0xbb, 0xdf, 0x1d, 0x5d, 0x30, 0x21, 0xd0, 0x5f, 0x18, 0xfb, 0xdd,
0xf2, 0xb7, 0x54, 0x78, 0xa6, 0x62, 0x2e, 0xce, 0x0b, 0x28, 0x7f, 0x03, 0xb7, 0x8e, 0x51, 0xe7,
0xe1, 0x52, 0x71, 0x57, 0xfe, 0x73, 0xa1, 0xfb, 0x7f, 0xae, 0xa7, 0x37, 0x48, 0x6f, 0xd5, 0x49,
0x69, 0x01, 0x0e, 0x16, 0x0f, 0xf2, 0x28, 0xf0, 0xfe, 0x47, 0xfb, 0x4a, 0xa1, 0x0f, 0x9f, 0xbc,
0xf8, 0xec, 0x9c, 0xab, 0x8b, 0xc9, 0x28, 0xf1, 0xee, 0xbd, 0xe6, 0xbe, 0xcf, 0x5f, 0x2b, 0x74,
0x2f, 0x7a, 0xc6, 0xe1, 0x03, 0x8f, 0x4b, 0x15, 0xf3, 0xd1, 0x44, 0xa1, 0xd7, 0xcb, 0x5a, 0xef,
0xe9, 0x90, 0x3d, 0x8d, 0x7e, 0x34, 0x1a, 0xd5, 0xb4, 0xf8, 0xe8, 0xef, 0x00, 0x00, 0x00, 0xff,
0xff, 0x3a, 0x9f, 0x25, 0xdf, 0x18, 0x0d, 0x00, 0x00,
// 922 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0x4f, 0x6f, 0x1b, 0x45,
0x14, 0xcf, 0x66, 0x13, 0x57, 0x7e, 0x76, 0xa3, 0x64, 0x52, 0x22, 0xcb, 0x50, 0x35, 0x59, 0xfa,
0xc7, 0x54, 0x60, 0x23, 0x57, 0x05, 0x09, 0x09, 0xa1, 0x26, 0x86, 0xc8, 0x42, 0x8d, 0xa2, 0x69,
0x05, 0xa2, 0x12, 0x8a, 0xc6, 0xde, 0xe7, 0x64, 0xc4, 0xee, 0xec, 0x76, 0x67, 0x5c, 0xe1, 0x5e,
0x10, 0x9f, 0x80, 0x03, 0x12, 0x5f, 0x84, 0x03, 0x27, 0xbe, 0x0e, 0x07, 0xee, 0xdc, 0xd1, 0xce,
0xec, 0xae, 0x77, 0xed, 0xad, 0xed, 0x28, 0xa0, 0x5c, 0x7a, 0xf3, 0x7b, 0xfb, 0xfe, 0xfe, 0xde,
0xef, 0xbd, 0x31, 0xec, 0x72, 0xe1, 0xe2, 0x8f, 0x67, 0x12, 0xa3, 0x57, 0x7c, 0x88, 0xed, 0x30,
0x0a, 0x54, 0x40, 0x88, 0xcf, 0xbd, 0x57, 0x63, 0x69, 0xa4, 0xb6, 0xb6, 0x68, 0xd6, 0x87, 0x81,
0xef, 0x07, 0xc2, 0xe8, 0x9a, 0x5b, 0x5c, 0x28, 0x8c, 0x04, 0xf3, 0x12, 0xb9, 0x9e, 0xf7, 0x70,
0x7e, 0x82, 0x5d, 0x8a, 0xe7, 0x5c, 0x2a, 0x8c, 0x4e, 0x02, 0x17, 0x29, 0xbe, 0x1c, 0xa3, 0x54,
0xe4, 0x63, 0xd8, 0x18, 0x30, 0x89, 0x0d, 0x6b, 0xdf, 0x6a, 0xd5, 0xba, 0xef, 0xb5, 0x0b, 0x59,
0x92, 0xf0, 0x4f, 0xe5, 0xf9, 0x21, 0x93, 0x48, 0xb5, 0x25, 0xf9, 0x04, 0x6e, 0x30, 0xd7, 0x8d,
0x50, 0xca, 0xc6, 0xfa, 0x02, 0xa7, 0x27, 0xc6, 0x86, 0xa6, 0xc6, 0xce, 0x2f, 0x16, 0xdc, 0x2a,
0x56, 0x20, 0xc3, 0x40, 0x48, 0x24, 0x8f, 0xa0, 0x22, 0x15, 0x53, 0x63, 0x99, 0x14, 0xf1, 0x6e,
0x69, 0xbc, 0x67, 0xda, 0x84, 0x26, 0xa6, 0xe4, 0x10, 0x6a, 0x5c, 0x70, 0x75, 0x16, 0xb2, 0x88,
0xf9, 0x69, 0x25, 0x07, 0xed, 0x19, 0x90, 0x12, 0x3c, 0xfa, 0x82, 0xab, 0x53, 0x6d, 0x48, 0x81,
0x67, 0xbf, 0x9d, 0xcf, 0x80, 0xf4, 0x63, 0x1c, 0xe3, 0xd0, 0x28, 0x53, 0x44, 0xee, 0xc2, 0x4d,
0x8d, 0xee, 0xe1, 0x98, 0x7b, 0x6e, 0xbf, 0x17, 0x57, 0x65, 0xb7, 0x6c, 0x5a, 0x54, 0x3a, 0x7f,
0x58, 0x50, 0xd5, 0xce, 0x7d, 0x31, 0x0a, 0xc8, 0x63, 0xd8, 0x8c, 0xeb, 0x32, 0x30, 0x6e, 0x75,
0xef, 0x94, 0x76, 0x30, 0xcd, 0x45, 0x8d, 0x35, 0x71, 0xa0, 0x9e, 0x8f, 0xaa, 0xbb, 0xb0, 0x69,
0x41, 0x47, 0x1a, 0x70, 0x43, 0xcb, 0xfd, 0x5e, 0xc3, 0xd6, 0x9f, 0x53, 0x91, 0xdc, 0x06, 0x30,
0x44, 0x11, 0xcc, 0xc7, 0xc6, 0xc6, 0xbe, 0xd5, 0xaa, 0xd2, 0xaa, 0xd6, 0x9c, 0x30, 0x1f, 0xc9,
0x1e, 0x54, 0x28, 0x32, 0x19, 0x88, 0xc6, 0xa6, 0xfe, 0x94, 0x48, 0xce, 0xcf, 0x16, 0xec, 0x16,
0xda, 0xbe, 0xca, 0x18, 0x1e, 0x1b, 0x27, 0x8c, 0x27, 0x60, 0xb7, 0x6a, 0xdd, 0xdb, 0xed, 0x79,
0x9a, 0xb6, 0x33, 0x9c, 0x68, 0x62, 0xec, 0xfc, 0x63, 0xc1, 0x8e, 0x69, 0x30, 0xfe, 0x94, 0x22,
0x5f, 0x6c, 0xc8, 0x9a, 0x6d, 0x28, 0x87, 0xc4, 0xfa, 0x1c, 0x12, 0x2e, 0x53, 0xec, 0x2c, 0x64,
0xea, 0x42, 0x36, 0xec, 0x7d, 0x3b, 0x76, 0x8c, 0x35, 0xa7, 0xb1, 0x22, 0xe6, 0x8a, 0x9a, 0x84,
0x98, 0x72, 0x65, 0x43, 0x57, 0x7a, 0x50, 0xda, 0xde, 0xd7, 0x38, 0xf9, 0x86, 0x79, 0x63, 0x3c,
0x65, 0x3c, 0xa2, 0x10, 0x7b, 0x19, 0xae, 0x90, 0x5e, 0x32, 0xaa, 0x34, 0xc8, 0xe6, 0xaa, 0x41,
0x6a, 0xda, 0x2d, 0x61, 0x9c, 0x0f, 0x24, 0xdf, 0xf6, 0x55, 0x90, 0x5f, 0x81, 0x3b, 0x8e, 0x07,
0x37, 0xa7, 0xe9, 0x8e, 0x7c, 0x77, 0xce, 0xc9, 0x2a, 0x21, 0xdc, 0xa7, 0x60, 0x47, 0xf8, 0x32,
0xd9, 0xa8, 0x7b, 0x65, 0xf3, 0x9c, 0x9b, 0x1c, 0x8d, 0x3d, 0x9c, 0xdf, 0x2d, 0xd8, 0x9b, 0x7e,
0x3a, 0x09, 0x14, 0x1f, 0xf1, 0x21, 0x53, 0x3c, 0x10, 0xff, 0x5b, 0x87, 0xa4, 0x05, 0xdb, 0x66,
0x2c, 0x23, 0xee, 0x61, 0x61, 0xfe, 0x5b, 0x5a, 0xff, 0x15, 0xf7, 0xd0, 0x90, 0x60, 0x0f, 0x2a,
0x22, 0x70, 0xb1, 0xdf, 0xd3, 0x9b, 0x62, 0xd3, 0x44, 0x72, 0x3e, 0x87, 0x77, 0xfa, 0x05, 0xcb,
0xcb, 0xdd, 0x81, 0xdf, 0x2c, 0xd8, 0x29, 0xf8, 0xeb, 0x7b, 0x70, 0xfd, 0xfd, 0x3a, 0xbf, 0x5a,
0xb0, 0x37, 0xdb, 0xd8, 0x55, 0xf8, 0xd6, 0x03, 0xc8, 0xe5, 0x34, 0xdb, 0x7e, 0xef, 0x8d, 0xdb,
0x9e, 0x47, 0x83, 0x56, 0x47, 0x59, 0x55, 0x7f, 0xa7, 0x67, 0xf3, 0x29, 0x2a, 0xb6, 0x12, 0x1d,
0xb3, 0xd3, 0xba, 0x7e, 0xa9, 0xd3, 0x7a, 0x07, 0x6a, 0x23, 0xc6, 0xbd, 0xb3, 0xc8, 0x9c, 0x40,
0x5b, 0x1f, 0x13, 0x88, 0x55, 0xe6, 0x0c, 0xa6, 0x34, 0xdf, 0xb8, 0x2c, 0xcd, 0x4b, 0x47, 0xb0,
0x59, 0x3a, 0x82, 0x0f, 0x61, 0xbb, 0x17, 0x05, 0x61, 0xe1, 0xc6, 0xe5, 0x8e, 0x98, 0x55, 0x38,
0x62, 0xdd, 0xbf, 0x2a, 0x50, 0x37, 0x7d, 0x98, 0x77, 0x9f, 0x0c, 0xa1, 0x9e, 0x7f, 0x2f, 0xc9,
0x83, 0xb2, 0x22, 0x4b, 0xde, 0xf4, 0x66, 0x6b, 0xb9, 0xa1, 0x61, 0x82, 0xb3, 0x46, 0xbe, 0x07,
0x98, 0xf6, 0x49, 0x56, 0xc3, 0xa1, 0x79, 0x7f, 0x99, 0x59, 0x16, 0x7e, 0x08, 0x5b, 0xc7, 0xa8,
0x72, 0xcf, 0x0d, 0xb9, 0xff, 0x46, 0xce, 0x14, 0x9e, 0xe1, 0xe6, 0x83, 0xa5, 0x76, 0x59, 0x12,
0x0f, 0x76, 0xd2, 0x24, 0xd3, 0x7d, 0xff, 0x60, 0x29, 0x37, 0xb3, 0x54, 0x0f, 0x57, 0x31, 0xcd,
0x21, 0xb6, 0xad, 0x6f, 0xdb, 0x24, 0x87, 0xdb, 0xc3, 0xc5, 0x80, 0xe4, 0x6f, 0x61, 0x73, 0xd1,
0xb6, 0x39, 0x6b, 0xe4, 0x14, 0xaa, 0x19, 0x69, 0xc8, 0xdd, 0xb2, 0xb8, 0xb3, 0x9c, 0x5a, 0x16,
0xf1, 0x05, 0x90, 0x63, 0x54, 0x47, 0x81, 0x1f, 0x06, 0x02, 0x85, 0x4a, 0xe6, 0xd0, 0x2c, 0x75,
0xfa, 0xd2, 0x0f, 0xd5, 0x64, 0x7e, 0xbe, 0xc9, 0xff, 0xa8, 0x99, 0x18, 0xce, 0x1a, 0xf9, 0x56,
0xc7, 0x7e, 0xce, 0x7d, 0x7c, 0xce, 0x87, 0x3f, 0x1c, 0x5d, 0x30, 0x21, 0xd0, 0x5b, 0x18, 0xfb,
0xfd, 0xe2, 0xb7, 0x44, 0x78, 0xa6, 0x22, 0x2e, 0xce, 0x73, 0x28, 0x7f, 0x07, 0xb7, 0x8e, 0x51,
0xe7, 0xe1, 0x52, 0xf1, 0xa1, 0xfc, 0xef, 0x42, 0x77, 0xff, 0xb4, 0x93, 0x1b, 0xa4, 0xb7, 0xea,
0xa4, 0xb0, 0x00, 0x07, 0x8b, 0x07, 0x79, 0xe4, 0xbb, 0x6f, 0xe7, 0x77, 0xcd, 0xf3, 0x3b, 0x7c,
0xf2, 0xe2, 0x8b, 0x73, 0xae, 0x2e, 0xc6, 0x83, 0xd8, 0xbb, 0xf3, 0x9a, 0x7b, 0x1e, 0x7f, 0xad,
0x70, 0x78, 0xd1, 0x31, 0x0e, 0x1f, 0xb9, 0x5c, 0xaa, 0x88, 0x0f, 0xc6, 0x0a, 0xdd, 0x4e, 0xda,
0x7a, 0x47, 0x87, 0xec, 0xe8, 0x01, 0x84, 0x83, 0x41, 0x45, 0x8b, 0x8f, 0xfe, 0x0d, 0x00, 0x00,
0xff, 0xff, 0x22, 0x93, 0x04, 0x69, 0x6a, 0x0d, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
@ -1229,6 +1229,7 @@ type IndexNodeClient interface {
//
// @return Status
BuildIndex(ctx context.Context, in *BuildIndexCmd, opts ...grpc.CallOption) (*commonpb.Status, error)
DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
GetComponentStates(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error)
GetTimeTickChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
GetStatisticsChannel(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
@ -1251,6 +1252,15 @@ func (c *indexNodeClient) BuildIndex(ctx context.Context, in *BuildIndexCmd, opt
return out, nil
}
func (c *indexNodeClient) DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
out := new(commonpb.Status)
err := c.cc.Invoke(ctx, "/milvus.proto.index.IndexNode/DropIndex", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *indexNodeClient) GetComponentStates(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error) {
out := new(internalpb2.ComponentStates)
err := c.cc.Invoke(ctx, "/milvus.proto.index.IndexNode/GetComponentStates", in, out, opts...)
@ -1287,6 +1297,7 @@ type IndexNodeServer interface {
//
// @return Status
BuildIndex(context.Context, *BuildIndexCmd) (*commonpb.Status, error)
DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error)
GetComponentStates(context.Context, *commonpb.Empty) (*internalpb2.ComponentStates, error)
GetTimeTickChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
GetStatisticsChannel(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
@ -1299,6 +1310,9 @@ type UnimplementedIndexNodeServer struct {
func (*UnimplementedIndexNodeServer) BuildIndex(ctx context.Context, req *BuildIndexCmd) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method BuildIndex not implemented")
}
func (*UnimplementedIndexNodeServer) DropIndex(ctx context.Context, req *DropIndexRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method DropIndex not implemented")
}
func (*UnimplementedIndexNodeServer) GetComponentStates(ctx context.Context, req *commonpb.Empty) (*internalpb2.ComponentStates, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetComponentStates not implemented")
}
@ -1331,6 +1345,24 @@ func _IndexNode_BuildIndex_Handler(srv interface{}, ctx context.Context, dec fun
return interceptor(ctx, in, info, handler)
}
func _IndexNode_DropIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(DropIndexRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(IndexNodeServer).DropIndex(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.index.IndexNode/DropIndex",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(IndexNodeServer).DropIndex(ctx, req.(*DropIndexRequest))
}
return interceptor(ctx, in, info, handler)
}
func _IndexNode_GetComponentStates_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty)
if err := dec(in); err != nil {
@ -1393,6 +1425,10 @@ var _IndexNode_serviceDesc = grpc.ServiceDesc{
MethodName: "BuildIndex",
Handler: _IndexNode_BuildIndex_Handler,
},
{
MethodName: "DropIndex",
Handler: _IndexNode_DropIndex_Handler,
},
{
MethodName: "GetComponentStates",
Handler: _IndexNode_GetComponentStates_Handler,

View File

@ -24,6 +24,7 @@ type IndexNodeInterface interface {
Service
Component
BuildIndex(req *indexpb.BuildIndexCmd) (*commonpb.Status, error)
DropIndex(req *indexpb.DropIndexRequest) (*commonpb.Status, error)
}
type IndexServiceInterface interface {