IndexCoord handle events correctly (#17878)

Signed-off-by: Cai.Zhang <cai.zhang@zilliz.com>
pull/18140/head
cai.zhang 2022-07-07 14:44:21 +08:00 committed by GitHub
parent 8f1ba6a0bb
commit 0ad15a22c8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
43 changed files with 3638 additions and 1213 deletions

View File

@ -198,9 +198,15 @@ indexCoord:
address: localhost address: localhost
port: 31000 port: 31000
gc:
interval: 600 # gc interval in seconds
indexNode: indexNode:
port: 21121 port: 21121
scheduler:
buildParallel: 1
dataCoord: dataCoord:
address: localhost address: localhost
port: 13333 port: 13333

View File

@ -130,7 +130,7 @@ type Server struct {
session *sessionutil.Session session *sessionutil.Session
dnEventCh <-chan *sessionutil.SessionEvent dnEventCh <-chan *sessionutil.SessionEvent
icEventCh <-chan *sessionutil.SessionEvent //icEventCh <-chan *sessionutil.SessionEvent
qcEventCh <-chan *sessionutil.SessionEvent qcEventCh <-chan *sessionutil.SessionEvent
rcEventCh <-chan *sessionutil.SessionEvent rcEventCh <-chan *sessionutil.SessionEvent
@ -433,22 +433,23 @@ func (s *Server) initServiceDiscovery() error {
// TODO implement rewatch logic // TODO implement rewatch logic
s.dnEventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1, nil) s.dnEventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1, nil)
icSessions, icRevision, err := s.session.GetSessions(typeutil.IndexCoordRole) //icSessions, icRevision, err := s.session.GetSessions(typeutil.IndexCoordRole)
if err != nil { //if err != nil {
log.Error("DataCoord get IndexCoord session failed", zap.Error(err)) // log.Error("DataCoord get IndexCoord session failed", zap.Error(err))
return err // return err
} //}
serverIDs := make([]UniqueID, 0, len(icSessions)) //serverIDs := make([]UniqueID, 0, len(icSessions))
for _, session := range icSessions { //for _, session := range icSessions {
serverIDs = append(serverIDs, session.ServerID) // serverIDs = append(serverIDs, session.ServerID)
} //}
s.icEventCh = s.session.WatchServices(typeutil.IndexCoordRole, icRevision+1, nil) //s.icEventCh = s.session.WatchServices(typeutil.IndexCoordRole, icRevision+1, nil)
qcSessions, qcRevision, err := s.session.GetSessions(typeutil.QueryCoordRole) qcSessions, qcRevision, err := s.session.GetSessions(typeutil.QueryCoordRole)
if err != nil { if err != nil {
log.Error("DataCoord get QueryCoord session failed", zap.Error(err)) log.Error("DataCoord get QueryCoord session failed", zap.Error(err))
return err return err
} }
serverIDs := make([]UniqueID, 0, len(qcSessions))
for _, session := range qcSessions { for _, session := range qcSessions {
serverIDs = append(serverIDs, session.ServerID) serverIDs = append(serverIDs, session.ServerID)
} }
@ -725,12 +726,12 @@ func (s *Server) watchService(ctx context.Context) {
}() }()
return return
} }
case event, ok := <-s.icEventCh: //case event, ok := <-s.icEventCh:
if !ok { // if !ok {
s.stopServiceWatch() // s.stopServiceWatch()
return // return
} // }
s.processSessionEvent(ctx, "IndexCoord", event) // s.processSessionEvent(ctx, "IndexCoord", event)
case event, ok := <-s.qcEventCh: case event, ok := <-s.qcEventCh:
if !ok { if !ok {
s.stopServiceWatch() s.stopServiceWatch()

View File

@ -732,68 +732,68 @@ func TestService_WatchServices(t *testing.T) {
assert.True(t, flag) assert.True(t, flag)
} }
func TestServer_watchCoord(t *testing.T) { //func TestServer_watchCoord(t *testing.T) {
Params.Init() // Params.Init()
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) // etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
assert.Nil(t, err) // assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) // etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
assert.NotNil(t, etcdKV) // assert.NotNil(t, etcdKV)
factory := dependency.NewDefaultFactory(true) // factory := dependency.NewDefaultFactory(true)
svr := CreateServer(context.TODO(), factory) // svr := CreateServer(context.TODO(), factory)
svr.session = &sessionutil.Session{ // svr.session = &sessionutil.Session{
TriggerKill: true, // TriggerKill: true,
} // }
svr.kvClient = etcdKV // svr.kvClient = etcdKV
//
dnCh := make(chan *sessionutil.SessionEvent) // dnCh := make(chan *sessionutil.SessionEvent)
icCh := make(chan *sessionutil.SessionEvent) // //icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent) // qcCh := make(chan *sessionutil.SessionEvent)
rcCh := make(chan *sessionutil.SessionEvent) // rcCh := make(chan *sessionutil.SessionEvent)
//
svr.dnEventCh = dnCh // svr.dnEventCh = dnCh
svr.icEventCh = icCh // //svr.icEventCh = icCh
svr.qcEventCh = qcCh // svr.qcEventCh = qcCh
svr.rcEventCh = rcCh // svr.rcEventCh = rcCh
//
segRefer, err := NewSegmentReferenceManager(etcdKV, nil) // segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err) // assert.NoError(t, err)
assert.NotNil(t, segRefer) // assert.NotNil(t, segRefer)
svr.segReferManager = segRefer // svr.segReferManager = segRefer
//
sc := make(chan os.Signal, 1) // sc := make(chan os.Signal, 1)
signal.Notify(sc, syscall.SIGINT) // signal.Notify(sc, syscall.SIGINT)
defer signal.Reset(syscall.SIGINT) // defer signal.Reset(syscall.SIGINT)
closed := false // closed := false
sigQuit := make(chan struct{}, 1) // sigQuit := make(chan struct{}, 1)
//
svr.serverLoopWg.Add(1) // svr.serverLoopWg.Add(1)
go func() { // go func() {
svr.watchService(context.Background()) // svr.watchService(context.Background())
}() // }()
//
go func() { // go func() {
<-sc // <-sc
closed = true // closed = true
sigQuit <- struct{}{} // sigQuit <- struct{}{}
}() // }()
//
icCh <- &sessionutil.SessionEvent{ // icCh <- &sessionutil.SessionEvent{
EventType: sessionutil.SessionAddEvent, // EventType: sessionutil.SessionAddEvent,
Session: &sessionutil.Session{ // Session: &sessionutil.Session{
ServerID: 1, // ServerID: 1,
}, // },
} // }
icCh <- &sessionutil.SessionEvent{ // icCh <- &sessionutil.SessionEvent{
EventType: sessionutil.SessionDelEvent, // EventType: sessionutil.SessionDelEvent,
Session: &sessionutil.Session{ // Session: &sessionutil.Session{
ServerID: 1, // ServerID: 1,
}, // },
} // }
close(icCh) // close(icCh)
<-sigQuit // <-sigQuit
svr.serverLoopWg.Wait() // svr.serverLoopWg.Wait()
assert.True(t, closed) // assert.True(t, closed)
} //}
func TestServer_watchQueryCoord(t *testing.T) { func TestServer_watchQueryCoord(t *testing.T) {
Params.Init() Params.Init()
@ -809,12 +809,12 @@ func TestServer_watchQueryCoord(t *testing.T) {
svr.kvClient = etcdKV svr.kvClient = etcdKV
dnCh := make(chan *sessionutil.SessionEvent) dnCh := make(chan *sessionutil.SessionEvent)
icCh := make(chan *sessionutil.SessionEvent) //icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent) qcCh := make(chan *sessionutil.SessionEvent)
rcCh := make(chan *sessionutil.SessionEvent) rcCh := make(chan *sessionutil.SessionEvent)
svr.dnEventCh = dnCh svr.dnEventCh = dnCh
svr.icEventCh = icCh //svr.icEventCh = icCh
svr.qcEventCh = qcCh svr.qcEventCh = qcCh
svr.rcEventCh = rcCh svr.rcEventCh = rcCh
@ -872,12 +872,12 @@ func TestServer_watchRootCoord(t *testing.T) {
svr.kvClient = etcdKV svr.kvClient = etcdKV
dnCh := make(chan *sessionutil.SessionEvent) dnCh := make(chan *sessionutil.SessionEvent)
icCh := make(chan *sessionutil.SessionEvent) //icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent) qcCh := make(chan *sessionutil.SessionEvent)
rcCh := make(chan *sessionutil.SessionEvent) rcCh := make(chan *sessionutil.SessionEvent)
svr.dnEventCh = dnCh svr.dnEventCh = dnCh
svr.icEventCh = icCh //svr.icEventCh = icCh
svr.qcEventCh = qcCh svr.qcEventCh = qcCh
svr.rcEventCh = rcCh svr.rcEventCh = rcCh

View File

@ -163,3 +163,16 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
} }
return ret.(*milvuspb.GetMetricsResponse), err return ret.(*milvuspb.GetMetricsResponse), err
} }
func (c *Client) GetTaskSlots(ctx context.Context, req *indexpb.GetTaskSlotsRequest) (*indexpb.GetTaskSlotsResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.(indexpb.IndexNodeClient).GetTaskSlots(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*indexpb.GetTaskSlotsResponse), err
}

View File

@ -187,6 +187,13 @@ func TestIndexNodeClient(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
}) })
t.Run("GetTaskSlots", func(t *testing.T) {
req := &indexpb.GetTaskSlotsRequest{}
resp, err := inc.GetTaskSlots(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
})
err = ins.Stop() err = ins.Stop()
assert.Nil(t, err) assert.Nil(t, err)

View File

@ -237,6 +237,10 @@ func (s *Server) CreateIndex(ctx context.Context, req *indexpb.CreateIndexReques
return s.indexnode.CreateIndex(ctx, req) return s.indexnode.CreateIndex(ctx, req)
} }
func (s *Server) GetTaskSlots(ctx context.Context, req *indexpb.GetTaskSlotsRequest) (*indexpb.GetTaskSlotsResponse, error) {
return s.indexnode.GetTaskSlots(ctx, req)
}
// GetMetrics gets the metrics info of IndexNode. // GetMetrics gets the metrics info of IndexNode.
func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return s.indexnode.GetMetrics(ctx, request) return s.indexnode.GetMetrics(ctx, request)

View File

@ -92,6 +92,13 @@ func TestIndexNodeServer(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
}) })
t.Run("GetTaskSlots", func(t *testing.T) {
req := &indexpb.GetTaskSlotsRequest{}
resp, err := server.GetTaskSlots(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
})
err = server.Stop() err = server.Stop()
assert.Nil(t, err) assert.Nil(t, err)
} }

View File

@ -21,6 +21,10 @@ import (
"fmt" "fmt"
) )
var (
ErrCompareVersion = errors.New("failed to save meta in etcd because version compare failure")
)
// errIndexNodeIsNotOnService return an error that the specified IndexNode is not exists. // errIndexNodeIsNotOnService return an error that the specified IndexNode is not exists.
func errIndexNodeIsNotOnService(id UniqueID) error { func errIndexNodeIsNotOnService(id UniqueID) error {
return fmt.Errorf("index node %d is not on service", id) return fmt.Errorf("index node %d is not on service", id)

View File

@ -0,0 +1,148 @@
package indexcoord
import (
"context"
"sync"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/storage"
"go.uber.org/zap"
)
type garbageCollector struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
gcFileDuration time.Duration
gcMetaDuration time.Duration
metaTable *metaTable
chunkManager storage.ChunkManager
}
func newGarbageCollector(ctx context.Context, meta *metaTable, chunkManager storage.ChunkManager) *garbageCollector {
ctx, cancel := context.WithCancel(ctx)
return &garbageCollector{
ctx: ctx,
cancel: cancel,
gcFileDuration: Params.IndexCoordCfg.GCInterval,
gcMetaDuration: time.Second * 10,
metaTable: meta,
chunkManager: chunkManager,
}
}
func (gc *garbageCollector) Start() {
gc.wg.Add(1)
go gc.recycleUnusedMeta()
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
}
func (gc *garbageCollector) Stop() {
gc.cancel()
gc.wg.Wait()
}
func (gc *garbageCollector) recycleUnusedMeta() {
defer gc.wg.Done()
log.Info("IndexCoord garbageCollector recycleUnusedMetaLoop start")
ticker := time.NewTicker(gc.gcMetaDuration)
defer ticker.Stop()
for {
select {
case <-gc.ctx.Done():
log.Info("IndexCoord garbageCollector recycleUnusedMetaLoop context has done")
return
case <-ticker.C:
metas := gc.metaTable.GetDeletedMetas()
for _, meta := range metas {
log.Info("index meta is deleted, recycle it", zap.Int64("buildID", meta.IndexBuildID),
zap.Int64("nodeID", meta.NodeID))
if meta.NodeID != 0 {
// wait for releasing reference lock
continue
}
if err := gc.metaTable.DeleteIndex(meta.IndexBuildID); err != nil {
log.Warn("delete index meta from etcd failed, wait to retry", zap.Int64("buildID", meta.IndexBuildID),
zap.Int64("nodeID", meta.NodeID), zap.Error(err))
continue
}
}
}
}
}
// recycleUnusedIndexFiles is used to delete those index files that no longer exist in the meta.
func (gc *garbageCollector) recycleUnusedIndexFiles() {
defer gc.wg.Done()
log.Info("IndexCoord garbageCollector start recycleUnusedIndexFiles loop")
ticker := time.NewTicker(gc.gcFileDuration)
defer ticker.Stop()
for {
select {
case <-gc.ctx.Done():
return
case <-ticker.C:
indexID2Files := gc.metaTable.GetBuildID2IndexFiles()
prefix := Params.IndexNodeCfg.IndexStorageRootPath + "/"
// list dir first
keys, err := gc.chunkManager.ListWithPrefix(prefix, false)
if err != nil {
log.Error("IndexCoord garbageCollector recycleUnusedIndexFiles list keys from chunk manager failed", zap.Error(err))
continue
}
for _, key := range keys {
buildID, err := parseBuildIDFromFilePath(key)
if err != nil {
log.Error("IndexCoord garbageCollector recycleUnusedIndexFiles parseIndexFileKey", zap.String("key", key), zap.Error(err))
continue
}
indexFiles, ok := indexID2Files[buildID]
if !ok {
// buildID no longer exists in meta, remove all index files
log.Info("IndexCoord garbageCollector recycleUnusedIndexFiles find meta has not exist, remove index files",
zap.Int64("buildID", buildID))
err = gc.chunkManager.RemoveWithPrefix(key)
if err != nil {
log.Warn("IndexCoord garbageCollector recycleUnusedIndexFiles remove index files failed",
zap.Int64("buildID", buildID), zap.String("prefix", key), zap.Error(err))
continue
}
continue
}
// Prevent IndexNode from being recycled as soon as the index file is written
if !gc.metaTable.CanBeRecycledIndexFiles(buildID) {
continue
}
// buildID still exists in meta, remove unnecessary index files
filesMap := make(map[string]bool)
for _, file := range indexFiles {
filesMap[file] = true
}
files, err := gc.chunkManager.ListWithPrefix(key, true)
if err != nil {
log.Warn("IndexCoord garbageCollector recycleUnusedIndexFiles list files failed",
zap.Int64("buildID", buildID), zap.String("prefix", key), zap.Error(err))
continue
}
for _, file := range files {
if _, ok := filesMap[file]; !ok {
if err = gc.chunkManager.Remove(file); err != nil {
log.Warn("IndexCoord garbageCollector recycleUnusedIndexFiles remove file failed",
zap.Int64("buildID", buildID), zap.String("file", file), zap.Error(err))
continue
}
}
}
}
}
}
}

View File

@ -0,0 +1,358 @@
package indexcoord
import (
"context"
"fmt"
"sync"
"testing"
"time"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
func TestGarbageCollector_Start(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{},
chunkManager: &chunkManagerMock{
removeWithPrefix: func(s string) error {
return nil
},
listWithPrefix: func(s string, recursive bool) ([]string, error) {
return []string{}, nil
},
remove: func(s string) error {
return nil
},
},
}
gc.Start()
gc.Stop()
}
func TestGarbageCollector_recycleUnusedIndexFiles(t *testing.T) {
t.Run("index not in meta and remove with prefix failed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
IndexFilePaths: []string{"file1", "file2", "file3"},
State: commonpb.IndexState_Finished,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return fmt.Errorf("error")
},
},
},
chunkManager: &chunkManagerMock{
removeWithPrefix: func(s string) error {
return fmt.Errorf("error")
},
listWithPrefix: func(s string, recursive bool) ([]string, error) {
if !recursive {
return []string{"a/b/1/", "a/b/2/"}, nil
}
return []string{"a/b/1/c", "a/b/2/d"}, nil
},
remove: func(s string) error {
return nil
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
t.Run("load dir failed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
IndexFilePaths: []string{"file1", "file2", "file3"},
State: commonpb.IndexState_Finished,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return fmt.Errorf("error")
},
},
},
chunkManager: &chunkManagerMock{
removeWithPrefix: func(s string) error {
return nil
},
listWithPrefix: func(s string, recursive bool) ([]string, error) {
if !recursive {
return nil, fmt.Errorf("error")
}
return []string{"a/b/1/c", "a/b/2/d"}, nil
},
remove: func(s string) error {
return nil
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
t.Run("parse failed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
IndexFilePaths: []string{"file1", "file2", "file3"},
State: commonpb.IndexState_Finished,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return fmt.Errorf("error")
},
},
},
chunkManager: &chunkManagerMock{
removeWithPrefix: func(s string) error {
return nil
},
listWithPrefix: func(s string, recursive bool) ([]string, error) {
if !recursive {
return []string{"a/b/c/"}, nil
}
return []string{"a/b/1/c", "a/b/2/d"}, nil
},
remove: func(s string) error {
return nil
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
t.Run("ListWithPrefix failed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
IndexFilePaths: []string{"file1", "file2", "file3"},
State: commonpb.IndexState_Finished,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return fmt.Errorf("error")
},
},
},
chunkManager: &chunkManagerMock{
removeWithPrefix: func(s string) error {
return nil
},
listWithPrefix: func(s string, recursive bool) ([]string, error) {
if !recursive {
return []string{"a/b/1/"}, nil
}
return nil, fmt.Errorf("error")
},
remove: func(s string) error {
return nil
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
t.Run("remove failed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
IndexFilePaths: []string{"file1", "file2", "file3"},
State: commonpb.IndexState_Finished,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return fmt.Errorf("error")
},
},
},
chunkManager: &chunkManagerMock{
removeWithPrefix: func(s string) error {
return nil
},
listWithPrefix: func(s string, recursive bool) ([]string, error) {
if !recursive {
return []string{"a/b/1/"}, nil
}
return []string{"a/b/1/c"}, nil
},
remove: func(s string) error {
return fmt.Errorf("error")
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
}
func TestIndexCoord_recycleUnusedMetaLoop(t *testing.T) {
t.Run("success", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
MarkDeleted: true,
NodeID: 0,
},
},
2: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 2,
MarkDeleted: false,
State: commonpb.IndexState_Finished,
NodeID: 0,
},
},
3: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 3,
MarkDeleted: false,
State: commonpb.IndexState_Finished,
NodeID: 1,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return nil
},
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedMeta()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
t.Run("remove meta failed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
gc := &garbageCollector{
ctx: ctx,
cancel: cancel,
wg: sync.WaitGroup{},
gcFileDuration: time.Millisecond * 300,
gcMetaDuration: time.Millisecond * 300,
metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
MarkDeleted: true,
NodeID: 0,
},
},
},
client: &mockETCDKV{
remove: func(s string) error {
return fmt.Errorf("error")
},
},
},
}
gc.wg.Add(1)
go gc.recycleUnusedMeta()
time.Sleep(time.Second)
cancel()
gc.wg.Wait()
})
}

View File

@ -0,0 +1,316 @@
package indexcoord
import (
"context"
"path"
"strconv"
"sync"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"go.uber.org/zap"
)
type indexBuilder struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
taskMutex sync.RWMutex
scheduleDuration time.Duration
// TODO @xiaocai2333: use priority queue
tasks map[int64]indexTaskState
notify chan bool
ic *IndexCoord
meta *metaTable
}
func newIndexBuilder(ctx context.Context, ic *IndexCoord, metaTable *metaTable, aliveNodes []UniqueID) *indexBuilder {
ctx, cancel := context.WithCancel(ctx)
ib := &indexBuilder{
ctx: ctx,
cancel: cancel,
meta: metaTable,
ic: ic,
tasks: make(map[int64]indexTaskState, 1024),
notify: make(chan bool, 1024),
scheduleDuration: time.Second * 10,
}
ib.reloadFromKV(aliveNodes)
return ib
}
func (ib *indexBuilder) Start() {
ib.wg.Add(1)
go ib.schedule()
}
func (ib *indexBuilder) Stop() {
ib.cancel()
close(ib.notify)
ib.wg.Wait()
}
func (ib *indexBuilder) reloadFromKV(aliveNodes []UniqueID) {
metas := ib.meta.GetAllIndexMeta()
for build, indexMeta := range metas {
// deleted, need to release lock and clean meta
if indexMeta.MarkDeleted {
if indexMeta.NodeID != 0 {
ib.tasks[build] = indexTaskDeleted
}
} else if indexMeta.State == commonpb.IndexState_Unissued && indexMeta.NodeID == 0 {
// unissued, need to acquire lock and assign task
ib.tasks[build] = indexTaskInit
} else if indexMeta.State == commonpb.IndexState_Unissued && indexMeta.NodeID != 0 {
// retry, need to release lock and reassign task
// need to release reference lock
ib.tasks[build] = indexTaskRetry
} else if indexMeta.State == commonpb.IndexState_InProgress {
// need to check IndexNode is still alive.
alive := false
for _, nodeID := range aliveNodes {
if nodeID == indexMeta.NodeID {
alive = true
break
}
}
if !alive {
// IndexNode is down, need to retry
ib.tasks[build] = indexTaskRetry
} else {
// in_progress, nothing to do
ib.tasks[build] = indexTaskInProgress
}
} else if indexMeta.State == commonpb.IndexState_Finished || indexMeta.State == commonpb.IndexState_Failed {
if indexMeta.NodeID != 0 {
// task is done, but the lock has not been released, need to release.
ib.tasks[build] = indexTaskDone
}
// else: task is done, and lock has been released, no need to add to index builder.
}
}
}
func (ib *indexBuilder) enqueue(buildID UniqueID) {
// notify
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
ib.tasks[buildID] = indexTaskInit
// why use false?
ib.notify <- false
}
func (ib *indexBuilder) schedule() {
// receive notify
// time ticker
defer ib.wg.Done()
ticker := time.NewTicker(ib.scheduleDuration)
defer ticker.Stop()
for {
select {
case <-ib.ctx.Done():
log.Warn("index builder ctx done")
return
case _, ok := <-ib.notify:
if ok {
ib.taskMutex.Lock()
log.Info("index builder task schedule", zap.Int("task num", len(ib.tasks)))
for buildID := range ib.tasks {
ib.process(buildID)
}
ib.taskMutex.Unlock()
}
// !ok means indexBuild is closed.
case <-ticker.C:
ib.taskMutex.Lock()
for buildID := range ib.tasks {
ib.process(buildID)
}
ib.taskMutex.Unlock()
}
}
}
func (ib *indexBuilder) process(buildID UniqueID) {
state := ib.tasks[buildID]
log.Info("index task is processing", zap.Int64("buildID", buildID), zap.String("task state", state.String()))
meta, exist := ib.meta.GetMeta(buildID)
switch state {
case indexTaskInit:
// peek client
// if all IndexNodes are executing task, wait for one of them to finish the task.
nodeID, client := ib.ic.nodeManager.PeekClient(meta)
if client == nil {
log.Error("index builder peek client error, there is no available")
return
}
// update version and set nodeID
if err := ib.meta.UpdateVersion(buildID, nodeID); err != nil {
log.Error("index builder update index version failed", zap.Int64("build", buildID), zap.Error(err))
return
}
// acquire lock
if err := ib.ic.tryAcquireSegmentReferLock(ib.ctx, buildID, nodeID, []UniqueID{meta.indexMeta.Req.SegmentID}); err != nil {
log.Error("index builder acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry
return
}
req := &indexpb.CreateIndexRequest{
IndexBuildID: buildID,
IndexName: meta.indexMeta.Req.IndexName,
IndexID: meta.indexMeta.Req.IndexID,
Version: meta.indexMeta.IndexVersion + 1,
MetaPath: path.Join(indexFilePrefix, strconv.FormatInt(buildID, 10)),
DataPaths: meta.indexMeta.Req.DataPaths,
TypeParams: meta.indexMeta.Req.TypeParams,
IndexParams: meta.indexMeta.Req.IndexParams,
}
if err := ib.ic.assignTask(client, req); err != nil {
// need to release lock then reassign, so set task state to retry
log.Error("index builder assign task to IndexNode failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry
return
}
// update index meta state to InProgress
if err := ib.meta.BuildIndex(buildID); err != nil {
// need to release lock then reassign, so set task state to retry
log.Error("index builder update index meta to InProgress failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry
return
}
ib.tasks[buildID] = indexTaskInProgress
case indexTaskDone:
if err := ib.releaseLockAndResetNode(buildID, meta.indexMeta.NodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Error("index builder try to release reference lock failed", zap.Error(err))
return
}
delete(ib.tasks, buildID)
case indexTaskRetry:
if err := ib.releaseLockAndResetTask(buildID, meta.indexMeta.NodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Error("index builder try to release reference lock failed", zap.Error(err))
return
}
ib.tasks[buildID] = indexTaskInit
ib.notify <- false
case indexTaskDeleted:
if exist && meta.indexMeta.NodeID != 0 {
if err := ib.releaseLockAndResetNode(buildID, meta.indexMeta.NodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Error("index builder try to release reference lock failed", zap.Error(err))
return
}
}
// reset nodeID success, remove task.
delete(ib.tasks, buildID)
}
}
func (ib *indexBuilder) releaseLockAndResetNode(buildID UniqueID, nodeID UniqueID) error {
log.Info("release segment reference lock and reset nodeID", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Error("index builder try to release reference lock failed", zap.Error(err))
return err
}
if err := ib.meta.ResetNodeID(buildID); err != nil {
log.Error("index builder try to reset nodeID failed", zap.Error(err))
return err
}
log.Info("release segment reference lock and reset nodeID success", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return nil
}
func (ib *indexBuilder) releaseLockAndResetTask(buildID UniqueID, nodeID UniqueID) error {
log.Info("release segment reference lock and reset task", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
if nodeID != 0 {
if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Error("index builder try to release reference lock failed", zap.Error(err))
return err
}
}
if err := ib.meta.ResetMeta(buildID); err != nil {
log.Error("index builder try to reset task failed", zap.Error(err))
return err
}
log.Info("release segment reference lock and reset task success", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return nil
}
func (ib *indexBuilder) updateStateByMeta(meta *indexpb.IndexMeta) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
state, ok := ib.tasks[meta.IndexBuildID]
if !ok {
log.Warn("index task has been processed", zap.Int64("buildId", meta.IndexBuildID), zap.Any("meta", meta))
// no need to return error, this task must have been deleted.
return
}
if meta.State == commonpb.IndexState_Finished || meta.State == commonpb.IndexState_Failed {
ib.tasks[meta.IndexBuildID] = indexTaskDone
ib.notify <- false
log.Info("this task has been finished", zap.Int64("buildID", meta.IndexBuildID),
zap.String("original state", state.String()), zap.String("finish or failed", meta.State.String()))
return
}
// index state must be Unissued and NodeID is not zero
ib.tasks[meta.IndexBuildID] = indexTaskRetry
log.Info("this task need to retry", zap.Int64("buildID", meta.IndexBuildID),
zap.String("original state", state.String()), zap.String("index state", meta.State.String()),
zap.Int64("original nodeID", meta.NodeID))
ib.notify <- false
}
func (ib *indexBuilder) markTaskAsDeleted(buildID UniqueID) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
if _, ok := ib.tasks[buildID]; ok {
ib.tasks[buildID] = indexTaskDeleted
}
ib.notify <- false
}
func (ib *indexBuilder) nodeDown(nodeID UniqueID) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
metas := ib.meta.GetMetasByNodeID(nodeID)
for _, meta := range metas {
if ib.tasks[meta.indexMeta.IndexBuildID] != indexTaskDone {
ib.tasks[meta.indexMeta.IndexBuildID] = indexTaskRetry
}
}
ib.notify <- false
}
func (ib *indexBuilder) hasTask(buildID UniqueID) bool {
ib.taskMutex.RLock()
defer ib.taskMutex.RUnlock()
_, ok := ib.tasks[buildID]
return ok
}

View File

@ -0,0 +1,425 @@
package indexcoord
import (
"context"
"errors"
"testing"
"time"
clientv3 "go.etcd.io/etcd/client/v3"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/stretchr/testify/assert"
)
func createMetaTable() *metaTable {
return &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
State: commonpb.IndexState_Unissued,
NodeID: 1,
MarkDeleted: true,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
2: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 2,
State: commonpb.IndexState_Unissued,
NodeID: 0,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
3: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 3,
State: commonpb.IndexState_Unissued,
NodeID: 1,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
4: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 4,
State: commonpb.IndexState_InProgress,
NodeID: 1,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
5: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 5,
State: commonpb.IndexState_InProgress,
NodeID: 3,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
6: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 6,
State: commonpb.IndexState_Finished,
NodeID: 2,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
7: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 7,
State: commonpb.IndexState_Failed,
NodeID: 0,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
},
client: &mockETCDKV{
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return true, nil
},
},
}
}
func TestIndexBuilder(t *testing.T) {
ctx := context.Background()
ic := &IndexCoord{
loopCtx: ctx,
reqTimeoutInterval: time.Second * 5,
dataCoordClient: &DataCoordMock{
Fail: false,
Err: false,
},
nodeManager: &NodeManager{
nodeClients: map[UniqueID]types.IndexNode{
4: &indexnode.Mock{
Err: false,
Failure: false,
},
},
},
}
ib := newIndexBuilder(ctx, ic, createMetaTable(), []UniqueID{1, 2})
assert.Equal(t, 6, len(ib.tasks))
assert.Equal(t, indexTaskDeleted, ib.tasks[1])
assert.Equal(t, indexTaskInit, ib.tasks[2])
assert.Equal(t, indexTaskRetry, ib.tasks[3])
assert.Equal(t, indexTaskInProgress, ib.tasks[4])
assert.Equal(t, indexTaskRetry, ib.tasks[5])
assert.Equal(t, indexTaskDone, ib.tasks[6])
ib.scheduleDuration = time.Millisecond * 500
ib.Start()
t.Run("enqueue", func(t *testing.T) {
ib.meta.indexBuildID2Meta[8] = &Meta{
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 8,
State: commonpb.IndexState_Unissued,
NodeID: 0,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
}
ib.enqueue(8)
})
t.Run("node down", func(t *testing.T) {
ib.nodeDown(1)
})
t.Run("updateStateByMeta", func(t *testing.T) {
indexMeta := &indexpb.IndexMeta{
IndexBuildID: 2,
State: commonpb.IndexState_Finished,
NodeID: 3,
}
ib.updateStateByMeta(indexMeta)
indexMeta = &indexpb.IndexMeta{
IndexBuildID: 3,
State: commonpb.IndexState_Finished,
NodeID: 3,
}
ib.updateStateByMeta(indexMeta)
indexMeta = &indexpb.IndexMeta{
IndexBuildID: 4,
State: commonpb.IndexState_Failed,
NodeID: 3,
}
ib.updateStateByMeta(indexMeta)
indexMeta = &indexpb.IndexMeta{
IndexBuildID: 5,
State: commonpb.IndexState_Unissued,
NodeID: 3,
}
ib.updateStateByMeta(indexMeta)
indexMeta = &indexpb.IndexMeta{
IndexBuildID: 8,
State: commonpb.IndexState_Finished,
NodeID: 3,
}
ib.updateStateByMeta(indexMeta)
for {
ib.taskMutex.Lock()
if len(ib.tasks) == 1 {
ib.taskMutex.Unlock()
break
}
ib.taskMutex.Unlock()
time.Sleep(time.Second)
}
ib.taskMutex.RLock()
assert.Equal(t, indexTaskInProgress, ib.tasks[5])
ib.taskMutex.RUnlock()
})
ib.Stop()
t.Run("save meta error", func(t *testing.T) {
mt := createMetaTable()
mt.client = &mockETCDKV{
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return true, errors.New("error")
},
}
ib2 := newIndexBuilder(ctx, ic, mt, []UniqueID{1, 2})
ib2.scheduleDuration = time.Millisecond * 500
ib2.Start()
time.Sleep(time.Second)
ib2.Stop()
assert.Equal(t, 6, len(ib2.tasks))
assert.Equal(t, indexTaskDeleted, ib2.tasks[1])
assert.Equal(t, indexTaskInit, ib2.tasks[2])
assert.Equal(t, indexTaskRetry, ib2.tasks[3])
assert.Equal(t, indexTaskInProgress, ib2.tasks[4])
assert.Equal(t, indexTaskRetry, ib2.tasks[5])
assert.Equal(t, indexTaskDone, ib2.tasks[6])
})
}
func TestIndexBuilder_Error(t *testing.T) {
ctx := context.Background()
t.Run("PeekClient fail", func(t *testing.T) {
ic := &IndexCoord{
loopCtx: ctx,
reqTimeoutInterval: time.Second * 5,
dataCoordClient: &DataCoordMock{
Fail: false,
Err: false,
},
nodeManager: &NodeManager{},
}
mt := &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
State: commonpb.IndexState_Unissued,
NodeID: 0,
MarkDeleted: false,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
},
client: &mockETCDKV{
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return true, nil
},
},
}
ib := newIndexBuilder(ic.loopCtx, ic, mt, []UniqueID{})
ib.scheduleDuration = time.Millisecond * 500
ib.Start()
time.Sleep(time.Second)
ib.Stop()
})
t.Run("update version fail", func(t *testing.T) {
ic := &IndexCoord{
loopCtx: ctx,
reqTimeoutInterval: time.Second * 5,
dataCoordClient: &DataCoordMock{
Fail: false,
Err: false,
},
nodeManager: &NodeManager{
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
Err: false,
Failure: false,
},
},
},
}
mt := &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
State: commonpb.IndexState_Unissued,
NodeID: 0,
MarkDeleted: false,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
},
client: &mockETCDKV{
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return true, errors.New("error")
},
},
}
ib := newIndexBuilder(ic.loopCtx, ic, mt, []UniqueID{})
ib.scheduleDuration = time.Second
ib.Start()
time.Sleep(time.Second)
ib.Stop()
})
t.Run("acquire lock fail", func(t *testing.T) {
ic := &IndexCoord{
loopCtx: ctx,
reqTimeoutInterval: time.Second * 5,
dataCoordClient: &DataCoordMock{
Fail: false,
Err: true,
},
nodeManager: &NodeManager{
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
Err: false,
Failure: false,
},
},
},
}
mt := &metaTable{
indexBuildID2Meta: map[UniqueID]*Meta{
1: {
indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1,
State: commonpb.IndexState_Unissued,
NodeID: 0,
MarkDeleted: false,
Req: &indexpb.BuildIndexRequest{
NumRows: 100,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
},
},
},
},
client: &mockETCDKV{
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return true, nil
},
},
}
ib := newIndexBuilder(ic.loopCtx, ic, mt, []UniqueID{})
ib.scheduleDuration = time.Second
ib.Start()
time.Sleep(time.Second)
ib.Stop()
})
}

View File

@ -21,7 +21,6 @@ import (
"errors" "errors"
"math/rand" "math/rand"
"os" "os"
"path"
"strconv" "strconv"
"sync" "sync"
"sync/atomic" "sync/atomic"
@ -85,8 +84,10 @@ type IndexCoord struct {
etcdCli *clientv3.Client etcdCli *clientv3.Client
chunkManager storage.ChunkManager chunkManager storage.ChunkManager
metaTable *metaTable metaTable *metaTable
nodeManager *NodeManager nodeManager *NodeManager
indexBuilder *indexBuilder
garbageCollector *garbageCollector
metricsCacheManager *metricsinfo.MetricsCacheManager metricsCacheManager *metricsinfo.MetricsCacheManager
@ -96,9 +97,6 @@ type IndexCoord struct {
startOnce sync.Once startOnce sync.Once
reqTimeoutInterval time.Duration reqTimeoutInterval time.Duration
durationInterval time.Duration
assignTaskInterval time.Duration
taskLimit int
dataCoordClient types.DataCoord dataCoordClient types.DataCoord
@ -118,9 +116,6 @@ func NewIndexCoord(ctx context.Context, factory dependency.Factory) (*IndexCoord
loopCtx: ctx1, loopCtx: ctx1,
loopCancel: cancel, loopCancel: cancel,
reqTimeoutInterval: time.Second * 10, reqTimeoutInterval: time.Second * 10,
durationInterval: time.Second * 10,
assignTaskInterval: time.Second * 1,
taskLimit: 20,
factory: factory, factory: factory,
} }
i.UpdateStateCode(internalpb.StateCode_Abnormal) i.UpdateStateCode(internalpb.StateCode_Abnormal)
@ -174,11 +169,11 @@ func (i *IndexCoord) Init() error {
connectEtcdFn := func() error { connectEtcdFn := func() error {
etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.EtcdCfg.MetaRootPath) etcdKV := etcdkv.NewEtcdKV(i.etcdCli, Params.EtcdCfg.MetaRootPath)
metakv, err := NewMetaTable(etcdKV) metaTable, err := NewMetaTable(etcdKV)
if err != nil { if err != nil {
return err return err
} }
i.metaTable = metakv i.metaTable = metaTable
return err return err
} }
log.Debug("IndexCoord try to connect etcd") log.Debug("IndexCoord try to connect etcd")
@ -188,6 +183,7 @@ func (i *IndexCoord) Init() error {
initErr = err initErr = err
return return
} }
log.Debug("IndexCoord try to connect etcd success") log.Debug("IndexCoord try to connect etcd success")
i.nodeManager = NewNodeManager(i.loopCtx) i.nodeManager = NewNodeManager(i.loopCtx)
@ -198,23 +194,22 @@ func (i *IndexCoord) Init() error {
initErr = err initErr = err
return return
} }
aliveNodeID := make([]UniqueID, 0)
for _, session := range sessions { for _, session := range sessions {
session := session session := session
aliveNodeID = append(aliveNodeID, session.ServerID)
go func() { go func() {
if err := i.nodeManager.AddNode(session.ServerID, session.Address); err != nil { if err := i.nodeManager.AddNode(session.ServerID, session.Address); err != nil {
log.Error("IndexCoord", zap.Int64("ServerID", session.ServerID), log.Error("IndexCoord", zap.Int64("ServerID", session.ServerID),
zap.Error(err)) zap.Error(err))
} }
}() }()
} }
log.Debug("IndexCoord", zap.Int("IndexNode number", len(i.nodeManager.nodeClients))) log.Debug("IndexCoord", zap.Int("IndexNode number", len(i.nodeManager.nodeClients)))
i.indexBuilder = newIndexBuilder(i.loopCtx, i, i.metaTable, aliveNodeID)
// TODO silverxia add Rewatch logic // TODO silverxia add Rewatch logic
i.eventChan = i.session.WatchServices(typeutil.IndexNodeRole, revision+1, nil) i.eventChan = i.session.WatchServices(typeutil.IndexNodeRole, revision+1, nil)
nodeTasks := i.metaTable.GetNodeTaskStats()
for nodeID, taskNum := range nodeTasks {
i.nodeManager.pq.UpdatePriority(nodeID, taskNum)
}
//init idAllocator //init idAllocator
kvRootPath := Params.EtcdCfg.KvRootPath kvRootPath := Params.EtcdCfg.KvRootPath
@ -236,6 +231,7 @@ func (i *IndexCoord) Init() error {
log.Debug("IndexCoord new minio chunkManager success") log.Debug("IndexCoord new minio chunkManager success")
i.chunkManager = chunkManager i.chunkManager = chunkManager
i.garbageCollector = newGarbageCollector(i.loopCtx, i.metaTable, i.chunkManager)
i.sched, err = NewTaskScheduler(i.loopCtx, i.idAllocator, i.chunkManager, i.metaTable) i.sched, err = NewTaskScheduler(i.loopCtx, i.idAllocator, i.chunkManager, i.metaTable)
if err != nil { if err != nil {
log.Error("IndexCoord new task scheduler failed", zap.Error(err)) log.Error("IndexCoord new task scheduler failed", zap.Error(err))
@ -259,12 +255,6 @@ func (i *IndexCoord) Start() error {
i.loopWg.Add(1) i.loopWg.Add(1)
go i.tsLoop() go i.tsLoop()
i.loopWg.Add(1)
go i.recycleUnusedIndexFiles()
i.loopWg.Add(1)
go i.assignTaskLoop()
i.loopWg.Add(1) i.loopWg.Add(1)
go i.watchNodeLoop() go i.watchNodeLoop()
@ -273,6 +263,9 @@ func (i *IndexCoord) Start() error {
startErr = i.sched.Start() startErr = i.sched.Start()
i.indexBuilder.Start()
i.garbageCollector.Start()
i.UpdateStateCode(internalpb.StateCode_Healthy) i.UpdateStateCode(internalpb.StateCode_Healthy)
}) })
// Start callbacks // Start callbacks
@ -303,9 +296,17 @@ func (i *IndexCoord) Stop() error {
i.sched.Close() i.sched.Close()
log.Info("close the task scheduler of IndexCoord") log.Info("close the task scheduler of IndexCoord")
} }
i.loopWg.Wait() i.loopWg.Wait()
if i.indexBuilder != nil {
i.indexBuilder.Stop()
log.Info("stop the index builder of IndexCoord")
}
if i.garbageCollector != nil {
i.garbageCollector.Stop()
log.Info("stop the garbage collector of IndexCoord")
}
for _, cb := range i.closeCallbacks { for _, cb := range i.closeCallbacks {
cb() cb()
} }
@ -472,6 +473,7 @@ func (i *IndexCoord) BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequ
metrics.IndexCoordIndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc() metrics.IndexCoordIndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
return ret, nil return ret, nil
} }
i.indexBuilder.enqueue(t.indexBuildID)
sp.SetTag("IndexCoord-IndexBuildID", strconv.FormatInt(t.indexBuildID, 10)) sp.SetTag("IndexCoord-IndexBuildID", strconv.FormatInt(t.indexBuildID, 10))
ret.Status.ErrorCode = commonpb.ErrorCode_Success ret.Status.ErrorCode = commonpb.ErrorCode_Success
ret.IndexBuildID = t.indexBuildID ret.IndexBuildID = t.indexBuildID
@ -548,19 +550,16 @@ func (i *IndexCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexReques
ret := &commonpb.Status{ ret := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success, ErrorCode: commonpb.ErrorCode_Success,
} }
err := i.metaTable.MarkIndexAsDeleted(req.IndexID) buildIDs, err := i.metaTable.MarkIndexAsDeleted(req.IndexID)
//no need do this. IndexNode finds that the task has been deleted, still changes the task status to finished, and writes back to etcd
//nodeTasks, err := i.metaTable.MarkIndexAsDeleted(req.IndexID)
//defer func() {
// for nodeID, taskNum := range nodeTasks {
// i.nodeManager.pq.IncPriority(nodeID, taskNum*-1)
// }
//}()
if err != nil { if err != nil {
ret.ErrorCode = commonpb.ErrorCode_UnexpectedError ret.ErrorCode = commonpb.ErrorCode_UnexpectedError
ret.Reason = err.Error() ret.Reason = err.Error()
return ret, nil return ret, nil
} }
log.Info("these buildIDs has been deleted", zap.Int64("indexID", req.IndexID), zap.Int64s("buildIDs", buildIDs))
for _, buildID := range buildIDs {
i.indexBuilder.markTaskAsDeleted(buildID)
}
defer func() { defer func() {
go func() { go func() {
@ -608,6 +607,9 @@ func (i *IndexCoord) RemoveIndex(ctx context.Context, req *indexpb.RemoveIndexRe
ret.Reason = err.Error() ret.Reason = err.Error()
return ret, nil return ret, nil
} }
for _, buildID := range req.BuildIDs {
i.indexBuilder.markTaskAsDeleted(buildID)
}
return ret, nil return ret, nil
} }
@ -752,58 +754,6 @@ func (i *IndexCoord) tsLoop() {
} }
} }
// recycleUnusedIndexFiles is used to delete useless index files, including lower version index files and index files
// corresponding to the deleted index.
func (i *IndexCoord) recycleUnusedIndexFiles() {
ctx, cancel := context.WithCancel(i.loopCtx)
defer cancel()
defer i.loopWg.Done()
timeTicker := time.NewTicker(i.durationInterval)
log.Debug("IndexCoord start recycleUnusedIndexFiles loop")
for {
select {
case <-ctx.Done():
return
case <-timeTicker.C:
metas := i.metaTable.GetUnusedIndexFiles(i.taskLimit)
for _, meta := range metas {
if meta.indexMeta.MarkDeleted {
unusedIndexFilePathPrefix := Params.IndexCoordCfg.IndexStorageRootPath + "/" + strconv.Itoa(int(meta.indexMeta.IndexBuildID))
log.Debug("IndexCoord recycleUnusedIndexFiles",
zap.Int64("Recycle the index files for deleted index with indexBuildID", meta.indexMeta.IndexBuildID))
if err := i.chunkManager.RemoveWithPrefix(unusedIndexFilePathPrefix); err != nil {
log.Error("IndexCoord recycleUnusedIndexFiles Remove index files failed",
zap.Bool("MarkDeleted", true), zap.Error(err))
}
i.metaTable.DeleteIndex(meta.indexMeta.IndexBuildID)
log.Debug("IndexCoord recycleUnusedIndexFiles",
zap.Int64("Recycle the index files successfully for deleted index with indexBuildID", meta.indexMeta.IndexBuildID))
} else {
log.Debug("IndexCoord recycleUnusedIndexFiles",
zap.Int64("Recycle the low version index files of the index with indexBuildID", meta.indexMeta.IndexBuildID),
zap.Int64("indexMeta version", meta.indexMeta.Version))
for j := 1; j < int(meta.indexMeta.Version); j++ {
unusedIndexFilePathPrefix := Params.IndexCoordCfg.IndexStorageRootPath + "/" + strconv.Itoa(int(meta.indexMeta.IndexBuildID)) + "/" + strconv.Itoa(j)
if err := i.chunkManager.RemoveWithPrefix(unusedIndexFilePathPrefix); err != nil {
log.Error("IndexCoord recycleUnusedIndexFiles Remove index files failed",
zap.Bool("MarkDeleted", false), zap.Error(err))
}
}
if err := i.metaTable.UpdateRecycleState(meta.indexMeta.IndexBuildID); err != nil {
log.Error("IndexCoord recycleUnusedIndexFiles UpdateRecycleState failed", zap.Error(err))
}
log.Debug("IndexCoord recycleUnusedIndexFiles",
zap.Int64("Recycle the low version index files successfully of the index with indexBuildID", meta.indexMeta.IndexBuildID))
}
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.RecycledIndexTaskLabel).Inc()
}
}
}
}
// watchNodeLoop is used to monitor IndexNode going online and offline. // watchNodeLoop is used to monitor IndexNode going online and offline.
//go:norace //go:norace
// fix datarace in unittest // fix datarace in unittest
@ -850,6 +800,8 @@ func (i *IndexCoord) watchNodeLoop() {
serverID := event.Session.ServerID serverID := event.Session.ServerID
log.Debug("IndexCoord watchNodeLoop SessionDelEvent", zap.Int64("serverID", serverID)) log.Debug("IndexCoord watchNodeLoop SessionDelEvent", zap.Int64("serverID", serverID))
i.nodeManager.RemoveNode(serverID) i.nodeManager.RemoveNode(serverID)
// remove tasks on nodeID
i.indexBuilder.nodeDown(serverID)
i.metricsCacheManager.InvalidateSystemInfoMetrics() i.metricsCacheManager.InvalidateSystemInfoMetrics()
} }
} }
@ -864,7 +816,7 @@ func (i *IndexCoord) watchMetaLoop() {
defer i.loopWg.Done() defer i.loopWg.Done()
log.Debug("IndexCoord watchMetaLoop start") log.Debug("IndexCoord watchMetaLoop start")
watchChan := i.metaTable.client.WatchWithRevision(indexFilePrefix, i.metaTable.revision) watchChan := i.metaTable.client.WatchWithRevision(indexFilePrefix, i.metaTable.etcdRevision)
for { for {
select { select {
case <-ctx.Done(): case <-ctx.Done():
@ -895,7 +847,6 @@ func (i *IndexCoord) watchMetaLoop() {
indexMeta := &indexpb.IndexMeta{} indexMeta := &indexpb.IndexMeta{}
err := proto.Unmarshal(event.Kv.Value, indexMeta) err := proto.Unmarshal(event.Kv.Value, indexMeta)
indexBuildID := indexMeta.IndexBuildID indexBuildID := indexMeta.IndexBuildID
log.Info("IndexCoord watchMetaLoop", zap.Int64("IndexBuildID", indexBuildID))
if err != nil { if err != nil {
log.Warn("IndexCoord unmarshal indexMeta failed", zap.Int64("IndexBuildID", indexBuildID), log.Warn("IndexCoord unmarshal indexMeta failed", zap.Int64("IndexBuildID", indexBuildID),
zap.Error(err)) zap.Error(err))
@ -903,57 +854,53 @@ func (i *IndexCoord) watchMetaLoop() {
} }
switch event.Type { switch event.Type {
case mvccpb.PUT: case mvccpb.PUT:
reload := i.metaTable.LoadMetaFromETCD(indexBuildID, eventRevision) log.Debug("IndexCoord watchMetaLoop", zap.Int64("IndexBuildID", indexBuildID),
log.Debug("IndexCoord watchMetaLoop PUT", zap.Int64("IndexBuildID", indexBuildID), zap.Bool("reload", reload)) zap.Int64("revision", eventRevision), zap.Any("indexMeta", indexMeta))
if reload { meta := &Meta{indexMeta: indexMeta, etcdVersion: eventRevision}
log.Debug("This task has finished or failed", zap.Int64("indexBuildID", indexBuildID), if i.metaTable.NeedUpdateMeta(meta) {
zap.Int64("Finish by IndexNode", indexMeta.NodeID), zap.String("index state", indexMeta.GetState().String()), log.Info("IndexCoord meta table update meta, update task state",
zap.Int64("The version of the task", indexMeta.Version)) zap.Int64("buildID", meta.indexMeta.IndexBuildID))
if err = i.tryReleaseSegmentReferLock(ctx, indexBuildID, []UniqueID{indexMeta.Req.SegmentID}); err != nil { // nothing to do, release reference lock.
panic(err) i.indexBuilder.updateStateByMeta(meta.indexMeta)
}
i.nodeManager.pq.IncPriority(indexMeta.NodeID, -1)
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.InProgressIndexTaskLabel).Dec()
if indexMeta.State == commonpb.IndexState_Finished {
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.FinishedIndexTaskLabel).Inc()
}
if indexMeta.State == commonpb.IndexState_Failed {
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.FailedIndexTaskLabel).Inc()
}
} }
case mvccpb.DELETE: case mvccpb.DELETE:
log.Debug("IndexCoord watchMetaLoop DELETE", zap.Int64("The meta has been deleted of indexBuildID", indexBuildID)) // why indexBuildID is zero in delete event?
log.Info("IndexCoord watchMetaLoop DELETE", zap.String("The meta has been deleted of key", string(event.Kv.Key)))
} }
} }
} }
} }
} }
func (i *IndexCoord) tryAcquireSegmentReferLock(ctx context.Context, buildID UniqueID, segIDs []UniqueID) error { func (i *IndexCoord) tryAcquireSegmentReferLock(ctx context.Context, buildID UniqueID, nodeID UniqueID, segIDs []UniqueID) error {
// IndexCoord use buildID instead of taskID. // IndexCoord use buildID instead of taskID.
log.Info("try to acquire segment reference lock", zap.Int64("buildID", buildID),
zap.Int64("ndoeID", nodeID), zap.Int64s("segIDs", segIDs))
status, err := i.dataCoordClient.AcquireSegmentLock(ctx, &datapb.AcquireSegmentLockRequest{ status, err := i.dataCoordClient.AcquireSegmentLock(ctx, &datapb.AcquireSegmentLockRequest{
TaskID: buildID, TaskID: buildID,
NodeID: i.session.ServerID, NodeID: nodeID,
SegmentIDs: segIDs, SegmentIDs: segIDs,
}) })
if err != nil { if err != nil {
log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64s("segIDs", segIDs), log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Error(err)) zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(err))
return err return err
} }
if status.ErrorCode != commonpb.ErrorCode_Success { if status.ErrorCode != commonpb.ErrorCode_Success {
log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64s("segIDs", segIDs), log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Error(errors.New(status.Reason))) zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(errors.New(status.Reason)))
return errors.New(status.Reason) return errors.New(status.Reason)
} }
log.Info("try to acquire segment reference lock success", zap.Int64("buildID", buildID),
zap.Int64("ndoeID", nodeID), zap.Int64s("segIDs", segIDs))
return nil return nil
} }
func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, buildID UniqueID, segIDs []UniqueID) error { func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, buildID UniqueID, nodeID UniqueID) error {
releaseLock := func() error { releaseLock := func() error {
status, err := i.dataCoordClient.ReleaseSegmentLock(ctx, &datapb.ReleaseSegmentLockRequest{ status, err := i.dataCoordClient.ReleaseSegmentLock(ctx, &datapb.ReleaseSegmentLockRequest{
TaskID: buildID, TaskID: buildID,
NodeID: i.session.ServerID, NodeID: nodeID,
}) })
if err != nil { if err != nil {
return err return err
@ -965,8 +912,8 @@ func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, buildID Uni
} }
err := retry.Do(ctx, releaseLock, retry.Attempts(100)) err := retry.Do(ctx, releaseLock, retry.Attempts(100))
if err != nil { if err != nil {
log.Error("IndexCoord try to release segment reference lock failed", zap.Int64s("segIDs", segIDs), log.Error("IndexCoord try to release segment reference lock failed", zap.Int64("buildID", buildID),
zap.Error(err)) zap.Int64("nodeID", nodeID), zap.Error(err))
return err return err
} }
return nil return nil
@ -974,99 +921,18 @@ func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, buildID Uni
// assignTask sends the index task to the IndexNode, it has a timeout interval, if the IndexNode doesn't respond within // assignTask sends the index task to the IndexNode, it has a timeout interval, if the IndexNode doesn't respond within
// the interval, it is considered that the task sending failed. // the interval, it is considered that the task sending failed.
func (i *IndexCoord) assignTask(builderClient types.IndexNode, req *indexpb.CreateIndexRequest) bool { func (i *IndexCoord) assignTask(builderClient types.IndexNode, req *indexpb.CreateIndexRequest) error {
ctx, cancel := context.WithTimeout(i.loopCtx, i.reqTimeoutInterval) ctx, cancel := context.WithTimeout(i.loopCtx, i.reqTimeoutInterval)
defer cancel() defer cancel()
resp, err := builderClient.CreateIndex(ctx, req) resp, err := builderClient.CreateIndex(ctx, req)
if err != nil { if err != nil {
log.Error("IndexCoord assignmentTasksLoop builderClient.CreateIndex failed", zap.Error(err)) log.Error("IndexCoord assignmentTasksLoop builderClient.CreateIndex failed", zap.Error(err))
return false return err
} }
if resp.ErrorCode != commonpb.ErrorCode_Success { if resp.ErrorCode != commonpb.ErrorCode_Success {
log.Error("IndexCoord assignmentTasksLoop builderClient.CreateIndex failed", zap.String("Reason", resp.Reason)) log.Error("IndexCoord assignmentTasksLoop builderClient.CreateIndex failed", zap.String("Reason", resp.Reason))
return false return errors.New(resp.Reason)
}
return true
}
// assignTaskLoop is used to assign index construction tasks.
func (i *IndexCoord) assignTaskLoop() {
ctx, cancel := context.WithCancel(i.loopCtx)
defer cancel()
defer i.loopWg.Done()
timeTicker := time.NewTicker(i.assignTaskInterval)
log.Debug("IndexCoord start assignTask loop")
for {
select {
case <-ctx.Done():
log.Debug("IndexCoord assignTaskLoop ctx Done")
return
case <-timeTicker.C:
serverIDs := i.nodeManager.ListNode()
if len(serverIDs) == 0 {
log.Warn("there is no indexnode online")
continue
}
metas := i.metaTable.GetUnassignedTasks(serverIDs)
// only log if we find unassigned tasks
if len(metas) != 0 {
log.Debug("IndexCoord find unassigned tasks ", zap.Int("Unassigned tasks number", len(metas)), zap.Int64s("Available IndexNode IDs", serverIDs))
}
for index, meta := range metas {
indexBuildID := meta.indexMeta.IndexBuildID
segID := meta.indexMeta.Req.SegmentID
nodeID, builderClient := i.nodeManager.PeekClient(meta)
if builderClient == nil && nodeID == -1 {
log.Warn("there is no indexnode online")
break
}
if builderClient == nil && nodeID == 0 {
log.Warn("The memory of all indexnodes does not meet the requirements")
continue
}
log.Debug("IndexCoord PeekClient success", zap.Int64("nodeID", nodeID))
if err := i.tryAcquireSegmentReferLock(ctx, indexBuildID, []UniqueID{segID}); err != nil {
log.Warn("IndexCoord try to acquire segment reference lock failed, maybe this segment has been compacted",
zap.Int64("segID", segID), zap.Int64("buildID", indexBuildID), zap.Error(err))
continue
}
if err := i.metaTable.UpdateVersion(indexBuildID); err != nil {
log.Warn("IndexCoord assignmentTasksLoop metaTable.UpdateVersion failed", zap.Error(err))
continue
}
log.Debug("The version of the task has been updated", zap.Int64("indexBuildID", indexBuildID))
req := &indexpb.CreateIndexRequest{
IndexBuildID: indexBuildID,
IndexName: meta.indexMeta.Req.IndexName,
IndexID: meta.indexMeta.Req.IndexID,
Version: meta.indexMeta.Version + 1,
MetaPath: path.Join(indexFilePrefix, strconv.FormatInt(indexBuildID, 10)),
DataPaths: meta.indexMeta.Req.DataPaths,
TypeParams: meta.indexMeta.Req.TypeParams,
IndexParams: meta.indexMeta.Req.IndexParams,
}
if !i.assignTask(builderClient, req) {
log.Warn("IndexCoord assignTask assign task to IndexNode failed")
continue
}
if err := i.metaTable.BuildIndex(indexBuildID, nodeID); err != nil {
log.Error("IndexCoord assignmentTasksLoop metaTable.BuildIndex failed", zap.Error(err))
break
}
log.Debug("This task has been assigned successfully", zap.Int64("indexBuildID", indexBuildID), zap.Int64("nodeID", nodeID))
i.nodeManager.pq.IncPriority(nodeID, 1)
if index > i.taskLimit {
break
}
}
}
} }
return nil
} }

View File

@ -21,6 +21,8 @@ import (
"errors" "errors"
"strconv" "strconv"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
@ -356,6 +358,8 @@ func (dcm *DataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.Re
}, nil }, nil
} }
// ChunkManagerMock is mock
// deprecated
type ChunkManagerMock struct { type ChunkManagerMock struct {
storage.ChunkManager storage.ChunkManager
@ -382,3 +386,53 @@ func (cmm *ChunkManagerMock) RemoveWithPrefix(prefix string) error {
} }
return nil return nil
} }
type mockETCDKV struct {
kv.MetaKv
remove func(string) error
watchWithRevision func(string, int64) clientv3.WatchChan
loadWithRevisionAndVersions func(string) ([]string, []string, []int64, int64, error)
compareVersionAndSwap func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error)
loadWithPrefix2 func(key string) ([]string, []string, []int64, error)
}
func (mk *mockETCDKV) Remove(key string) error {
return mk.remove(key)
}
func (mk *mockETCDKV) LoadWithRevisionAndVersions(prefix string) ([]string, []string, []int64, int64, error) {
return mk.loadWithRevisionAndVersions(prefix)
}
func (mk *mockETCDKV) CompareVersionAndSwap(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return mk.compareVersionAndSwap(key, version, target, opts...)
}
func (mk *mockETCDKV) LoadWithPrefix2(key string) ([]string, []string, []int64, error) {
return mk.loadWithPrefix2(key)
}
func (mk *mockETCDKV) WatchWithRevision(key string, revision int64) clientv3.WatchChan {
return mk.watchWithRevision(key, revision)
}
type chunkManagerMock struct {
storage.ChunkManager
removeWithPrefix func(string) error
listWithPrefix func(string, bool) ([]string, error)
remove func(string) error
}
func (cmm *chunkManagerMock) RemoveWithPrefix(prefix string) error {
return cmm.removeWithPrefix(prefix)
}
func (cmm *chunkManagerMock) ListWithPrefix(prefix string, recursive bool) ([]string, error) {
return cmm.listWithPrefix(prefix, recursive)
}
func (cmm *chunkManagerMock) Remove(key string) error {
return cmm.remove(key)
}

View File

@ -27,7 +27,6 @@ import (
"testing" "testing"
"time" "time"
"github.com/milvus-io/milvus/internal/kv"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -65,9 +64,6 @@ func TestIndexCoord(t *testing.T) {
ic, err := NewIndexCoord(ctx, factory) ic, err := NewIndexCoord(ctx, factory)
assert.Nil(t, err) assert.Nil(t, err)
ic.reqTimeoutInterval = time.Second * 10 ic.reqTimeoutInterval = time.Second * 10
ic.durationInterval = time.Second
ic.assignTaskInterval = 200 * time.Millisecond
ic.taskLimit = 20
dcm := &DataCoordMock{ dcm := &DataCoordMock{
Err: false, Err: false,
@ -163,6 +159,7 @@ func TestIndexCoord(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
indexBuildID = resp.IndexBuildID indexBuildID = resp.IndexBuildID
resp2, err := ic.BuildIndex(ctx, req) resp2, err := ic.BuildIndex(ctx, req)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
@ -199,7 +196,7 @@ func TestIndexCoord(t *testing.T) {
if resp.States[0].State == commonpb.IndexState_Finished { if resp.States[0].State == commonpb.IndexState_Finished {
break break
} }
time.Sleep(100 * time.Millisecond) time.Sleep(500 * time.Millisecond)
} }
}) })
@ -265,15 +262,6 @@ func TestIndexCoord(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode)
}) })
t.Run("Recycle IndexMeta", func(t *testing.T) {
indexMeta := ic.metaTable.GetIndexMetaByIndexBuildID(indexBuildID)
for indexMeta != nil {
log.Info("RecycleIndexMeta", zap.Any("meta", indexMeta))
indexMeta = ic.metaTable.GetIndexMetaByIndexBuildID(indexBuildID)
time.Sleep(100 * time.Millisecond)
}
})
t.Run("GetMetrics request without metricType", func(t *testing.T) { t.Run("GetMetrics request without metricType", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{ req := &milvuspb.GetMetricsRequest{
Request: "GetIndexCoordMetrics", Request: "GetIndexCoordMetrics",
@ -328,21 +316,6 @@ func TestIndexCoord_watchNodeLoop(t *testing.T) {
assert.True(t, closed) assert.True(t, closed)
} }
type mockEtcdKv struct {
kv.MetaKv
watchWithRevision func(string, int64) clientv3.WatchChan
loadWithRevisionAndVersions func(string) ([]string, []string, []int64, int64, error)
}
func (mek *mockEtcdKv) WatchWithRevision(key string, revision int64) clientv3.WatchChan {
return mek.watchWithRevision(key, revision)
}
func (mek *mockEtcdKv) LoadWithRevisionAndVersions(key string) ([]string, []string, []int64, int64, error) {
return mek.loadWithRevisionAndVersions(key)
}
func TestIndexCoord_watchMetaLoop(t *testing.T) { func TestIndexCoord_watchMetaLoop(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
ic := &IndexCoord{ ic := &IndexCoord{
@ -352,15 +325,15 @@ func TestIndexCoord_watchMetaLoop(t *testing.T) {
watchChan := make(chan clientv3.WatchResponse, 1024) watchChan := make(chan clientv3.WatchResponse, 1024)
client := &mockEtcdKv{ client := &mockETCDKV{
watchWithRevision: func(s string, i int64) clientv3.WatchChan { watchWithRevision: func(s string, i int64) clientv3.WatchChan {
return watchChan return watchChan
}, },
} }
mt := &metaTable{ mt := &metaTable{
client: client, client: client,
indexBuildID2Meta: map[UniqueID]Meta{}, indexBuildID2Meta: map[UniqueID]*Meta{},
revision: 0, etcdRevision: 0,
lock: sync.RWMutex{}, lock: sync.RWMutex{},
} }
ic.metaTable = mt ic.metaTable = mt
@ -376,7 +349,7 @@ func TestIndexCoord_watchMetaLoop(t *testing.T) {
}) })
t.Run("watch chan new meta table panic", func(t *testing.T) { t.Run("watch chan new meta table panic", func(t *testing.T) {
client = &mockEtcdKv{ client = &mockETCDKV{
watchWithRevision: func(s string, i int64) clientv3.WatchChan { watchWithRevision: func(s string, i int64) clientv3.WatchChan {
return watchChan return watchChan
}, },
@ -386,8 +359,8 @@ func TestIndexCoord_watchMetaLoop(t *testing.T) {
} }
mt = &metaTable{ mt = &metaTable{
client: client, client: client,
indexBuildID2Meta: map[UniqueID]Meta{}, indexBuildID2Meta: map[UniqueID]*Meta{},
revision: 0, etcdRevision: 0,
lock: sync.RWMutex{}, lock: sync.RWMutex{},
} }
ic.metaTable = mt ic.metaTable = mt
@ -401,7 +374,7 @@ func TestIndexCoord_watchMetaLoop(t *testing.T) {
t.Run("watch chan new meta success", func(t *testing.T) { t.Run("watch chan new meta success", func(t *testing.T) {
ic.loopWg = sync.WaitGroup{} ic.loopWg = sync.WaitGroup{}
client = &mockEtcdKv{ client = &mockETCDKV{
watchWithRevision: func(s string, i int64) clientv3.WatchChan { watchWithRevision: func(s string, i int64) clientv3.WatchChan {
return watchChan return watchChan
}, },
@ -411,8 +384,8 @@ func TestIndexCoord_watchMetaLoop(t *testing.T) {
} }
mt = &metaTable{ mt = &metaTable{
client: client, client: client,
indexBuildID2Meta: map[UniqueID]Meta{}, indexBuildID2Meta: map[UniqueID]*Meta{},
revision: 0, etcdRevision: 0,
lock: sync.RWMutex{}, lock: sync.RWMutex{},
} }
ic.metaTable = mt ic.metaTable = mt
@ -472,7 +445,7 @@ func TestIndexCoord_NotHealthy(t *testing.T) {
func TestIndexCoord_GetIndexFilePaths(t *testing.T) { func TestIndexCoord_GetIndexFilePaths(t *testing.T) {
ic := &IndexCoord{ ic := &IndexCoord{
metaTable: &metaTable{ metaTable: &metaTable{
indexBuildID2Meta: map[UniqueID]Meta{ indexBuildID2Meta: map[UniqueID]*Meta{
1: { 1: {
indexMeta: &indexpb.IndexMeta{ indexMeta: &indexpb.IndexMeta{
IndexBuildID: 1, IndexBuildID: 1,
@ -532,7 +505,7 @@ func Test_tryAcquireSegmentReferLock(t *testing.T) {
ic.chunkManager = cmm ic.chunkManager = cmm
t.Run("success", func(t *testing.T) { t.Run("success", func(t *testing.T) {
err := ic.tryAcquireSegmentReferLock(context.Background(), 1, []UniqueID{1}) err := ic.tryAcquireSegmentReferLock(context.Background(), 1, 1, []UniqueID{1})
assert.Nil(t, err) assert.Nil(t, err)
}) })
@ -542,7 +515,7 @@ func Test_tryAcquireSegmentReferLock(t *testing.T) {
Fail: false, Fail: false,
} }
ic.dataCoordClient = dcmE ic.dataCoordClient = dcmE
err := ic.tryAcquireSegmentReferLock(context.Background(), 1, []UniqueID{1}) err := ic.tryAcquireSegmentReferLock(context.Background(), 1, 1, []UniqueID{1})
assert.Error(t, err) assert.Error(t, err)
}) })
@ -552,7 +525,7 @@ func Test_tryAcquireSegmentReferLock(t *testing.T) {
Fail: true, Fail: true,
} }
ic.dataCoordClient = dcmF ic.dataCoordClient = dcmF
err := ic.tryAcquireSegmentReferLock(context.Background(), 1, []UniqueID{1}) err := ic.tryAcquireSegmentReferLock(context.Background(), 1, 1, []UniqueID{1})
assert.Error(t, err) assert.Error(t, err)
}) })
} }
@ -571,7 +544,7 @@ func Test_tryReleaseSegmentReferLock(t *testing.T) {
ic.dataCoordClient = dcm ic.dataCoordClient = dcm
t.Run("success", func(t *testing.T) { t.Run("success", func(t *testing.T) {
err := ic.tryReleaseSegmentReferLock(context.Background(), 1, []UniqueID{1}) err := ic.tryReleaseSegmentReferLock(context.Background(), 1, 1)
assert.NoError(t, err) assert.NoError(t, err)
}) })
} }
@ -579,6 +552,9 @@ func Test_tryReleaseSegmentReferLock(t *testing.T) {
func TestIndexCoord_RemoveIndex(t *testing.T) { func TestIndexCoord_RemoveIndex(t *testing.T) {
ic := &IndexCoord{ ic := &IndexCoord{
metaTable: &metaTable{}, metaTable: &metaTable{},
indexBuilder: &indexBuilder{
notify: make(chan bool, 10),
},
} }
ic.stateCode.Store(internalpb.StateCode_Healthy) ic.stateCode.Store(internalpb.StateCode_Healthy)
status, err := ic.RemoveIndex(context.Background(), &indexpb.RemoveIndexRequest{BuildIDs: []UniqueID{0}}) status, err := ic.RemoveIndex(context.Background(), &indexpb.RemoveIndexRequest{BuildIDs: []UniqueID{0}})

View File

@ -21,7 +21,6 @@ import (
"errors" "errors"
"fmt" "fmt"
"path" "path"
"sort"
"strconv" "strconv"
"sync" "sync"
@ -42,16 +41,16 @@ import (
// revision: The number of times IndexMeta has been changed in etcd. It's the same as Event.Kv.Version in etcd. // revision: The number of times IndexMeta has been changed in etcd. It's the same as Event.Kv.Version in etcd.
// indexMeta: A structure that records the state of the index defined by proto. // indexMeta: A structure that records the state of the index defined by proto.
type Meta struct { type Meta struct {
indexMeta *indexpb.IndexMeta indexMeta *indexpb.IndexMeta
indexVersion int64 etcdVersion int64
} }
// metaTable records the mapping of IndexBuildID to Meta. // metaTable records the mapping of IndexBuildID to Meta.
type metaTable struct { type metaTable struct {
client kv.MetaKv // client of a reliable kv service, i.e. etcd client client kv.MetaKv // client of a reliable kv service, i.e. etcd client
indexBuildID2Meta map[UniqueID]Meta // index build id to index meta indexBuildID2Meta map[UniqueID]*Meta // index build id to index meta
revision int64 etcdRevision int64
lock sync.RWMutex lock sync.RWMutex
} }
@ -72,7 +71,7 @@ func NewMetaTable(kv kv.MetaKv) (*metaTable, error) {
// reloadFromKV reloads the index meta from ETCD. // reloadFromKV reloads the index meta from ETCD.
func (mt *metaTable) reloadFromKV() error { func (mt *metaTable) reloadFromKV() error {
mt.indexBuildID2Meta = make(map[UniqueID]Meta) mt.indexBuildID2Meta = make(map[UniqueID]*Meta)
key := indexFilePrefix key := indexFilePrefix
log.Debug("IndexCoord metaTable LoadWithPrefix ", zap.String("prefix", key)) log.Debug("IndexCoord metaTable LoadWithPrefix ", zap.String("prefix", key))
@ -81,7 +80,7 @@ func (mt *metaTable) reloadFromKV() error {
return err return err
} }
mt.revision = revision mt.etcdRevision = revision
for i := 0; i < len(values); i++ { for i := 0; i < len(values); i++ {
indexMeta := indexpb.IndexMeta{} indexMeta := indexpb.IndexMeta{}
@ -91,10 +90,10 @@ func (mt *metaTable) reloadFromKV() error {
} }
meta := &Meta{ meta := &Meta{
indexMeta: &indexMeta, indexMeta: &indexMeta,
indexVersion: versions[i], etcdVersion: versions[i],
} }
mt.indexBuildID2Meta[indexMeta.IndexBuildID] = *meta mt.indexBuildID2Meta[indexMeta.IndexBuildID] = meta
} }
return nil return nil
} }
@ -107,19 +106,18 @@ func (mt *metaTable) saveIndexMeta(meta *Meta) error {
return err return err
} }
key := path.Join(indexFilePrefix, strconv.FormatInt(meta.indexMeta.IndexBuildID, 10)) key := path.Join(indexFilePrefix, strconv.FormatInt(meta.indexMeta.IndexBuildID, 10))
success, err := mt.client.CompareVersionAndSwap(key, meta.indexVersion, string(value)) success, err := mt.client.CompareVersionAndSwap(key, meta.etcdVersion, string(value))
if err != nil { if err != nil {
// TODO, we don't need to reload if it is just etcd error
log.Warn("failed to save index meta in etcd", zap.Int64("buildID", meta.indexMeta.IndexBuildID), zap.Error(err)) log.Warn("failed to save index meta in etcd", zap.Int64("buildID", meta.indexMeta.IndexBuildID), zap.Error(err))
return err return err
} }
if !success { if !success {
log.Warn("failed to save index meta in etcd because version compare failure", zap.Int64("buildID", meta.indexMeta.IndexBuildID), zap.Any("index", meta.indexMeta)) log.Warn("failed to save index meta in etcd because version compare failure", zap.Int64("buildID", meta.indexMeta.IndexBuildID), zap.Any("index", meta.indexMeta))
return fmt.Errorf("failed to save index meta in etcd, buildId: %d, source version: %d", meta.indexMeta.IndexBuildID, meta.indexVersion) return ErrCompareVersion
} }
meta.indexVersion = meta.indexVersion + 1 meta.etcdVersion = meta.etcdVersion + 1
mt.indexBuildID2Meta[meta.indexMeta.IndexBuildID] = *meta mt.indexBuildID2Meta[meta.indexMeta.IndexBuildID] = meta
log.Info("IndexCoord metaTable saveIndexMeta success", zap.Int64("buildID", meta.indexMeta.IndexBuildID), zap.Any("meta.revision", meta.indexVersion)) log.Info("IndexCoord metaTable saveIndexMeta success", zap.Int64("buildID", meta.indexMeta.IndexBuildID), zap.Int64("meta.revision", meta.etcdVersion))
return nil return nil
} }
@ -141,25 +139,36 @@ func (mt *metaTable) reloadMeta(indexBuildID UniqueID) (*Meta, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
//if im.State == commonpb.IndexState_Finished {
// return nil, nil
//}
m := &Meta{ m := &Meta{
indexVersion: version[0], etcdVersion: version[0],
indexMeta: im, indexMeta: im,
} }
log.Debug("reload meta from etcd success", zap.Int64("buildID", indexBuildID), zap.Any("indexMeta", im))
return m, nil return m, nil
} }
func (mt *metaTable) GetAllIndexMeta() map[int64]*indexpb.IndexMeta {
mt.lock.RLock()
defer mt.lock.RUnlock()
metas := map[int64]*indexpb.IndexMeta{}
for build, meta := range mt.indexBuildID2Meta {
metas[build] = proto.Clone(meta.indexMeta).(*indexpb.IndexMeta)
}
return metas
}
// AddIndex adds the index meta corresponding the indexBuildID to meta table. // AddIndex adds the index meta corresponding the indexBuildID to meta table.
func (mt *metaTable) AddIndex(indexBuildID UniqueID, req *indexpb.BuildIndexRequest) error { func (mt *metaTable) AddIndex(indexBuildID UniqueID, req *indexpb.BuildIndexRequest) error {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
_, ok := mt.indexBuildID2Meta[indexBuildID] _, ok := mt.indexBuildID2Meta[indexBuildID]
log.Debug("IndexCoord metaTable AddIndex", zap.Any("indexBuildID", indexBuildID), zap.Any(" index already exist", ok)) log.Debug("IndexCoord metaTable AddIndex", zap.Int64("indexBuildID", indexBuildID), zap.Bool(" index already exist", ok))
if ok { if ok {
return fmt.Errorf("index already exists with ID = %d", indexBuildID) log.Info("index already exists", zap.Int64("buildID", indexBuildID), zap.Int64("indexID", req.IndexID))
return nil
} }
meta := &Meta{ meta := &Meta{
indexMeta: &indexpb.IndexMeta{ indexMeta: &indexpb.IndexMeta{
@ -167,167 +176,247 @@ func (mt *metaTable) AddIndex(indexBuildID UniqueID, req *indexpb.BuildIndexRequ
IndexBuildID: indexBuildID, IndexBuildID: indexBuildID,
Req: req, Req: req,
NodeID: 0, NodeID: 0,
Version: 0, IndexVersion: 0,
}, },
indexVersion: 0, etcdVersion: 0,
} }
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.UnissuedIndexTaskLabel).Inc() metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.UnissuedIndexTaskLabel).Inc()
return mt.saveIndexMeta(meta) if err := mt.saveIndexMeta(meta); err != nil {
// no need to reload, no reason to compare version fail
log.Error("IndexCoord metaTable save index meta failed", zap.Int64("buildID", indexBuildID),
zap.Int64("indexID", req.IndexID), zap.Error(err))
return err
}
log.Info("IndexCoord metaTable AddIndex success", zap.Int64("buildID", indexBuildID))
return nil
} }
// BuildIndex set the index state to be InProgress. It means IndexNode is building the index. func (mt *metaTable) updateMeta(buildID UniqueID, updateFunc func(m *Meta) error) error {
func (mt *metaTable) BuildIndex(indexBuildID UniqueID, nodeID int64) error { meta, ok := mt.indexBuildID2Meta[buildID]
mt.lock.Lock()
defer mt.lock.Unlock()
log.Debug("IndexCoord metaTable BuildIndex")
meta, ok := mt.indexBuildID2Meta[indexBuildID]
if !ok { if !ok {
log.Error("IndexCoord metaTable BuildIndex index not exists", zap.Any("indexBuildID", indexBuildID)) log.Error("IndexCoord metaTable updateMeta index not exists", zap.Any("indexBuildID", buildID))
return fmt.Errorf("index not exists with ID = %d", indexBuildID) return fmt.Errorf("index not exists with ID = %d", buildID)
} }
clonedMeta := &Meta{
//if meta.indexMeta.State != commonpb.IndexState_Unissued { indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta),
// return fmt.Errorf("can not set lease key, index with ID = %d state is %d", indexBuildID, meta.indexMeta.State) etcdVersion: meta.etcdVersion,
//}
if meta.indexMeta.State == commonpb.IndexState_Finished || meta.indexMeta.State == commonpb.IndexState_Failed {
log.Debug("This index task has been finished", zap.Int64("indexBuildID", indexBuildID),
zap.Any("index state", meta.indexMeta.State))
return nil
} }
meta.indexMeta.NodeID = nodeID if err := updateFunc(clonedMeta); err != nil {
meta.indexMeta.State = commonpb.IndexState_InProgress if !errors.Is(err, ErrCompareVersion) {
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.UnissuedIndexTaskLabel).Dec() log.Error("IndexCoord metaTable updateMeta fail", zap.Int64("buildID", buildID), zap.Error(err))
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.InProgressIndexTaskLabel).Inc() return err
}
err := mt.saveIndexMeta(&meta)
if err != nil {
fn := func() error { fn := func() error {
m, err := mt.reloadMeta(meta.indexMeta.IndexBuildID) m, err := mt.reloadMeta(meta.indexMeta.IndexBuildID)
if m == nil { if err != nil {
return err return err
} }
m.indexMeta.NodeID = nodeID return updateFunc(m)
return mt.saveIndexMeta(m)
} }
err2 := retry.Do(context.TODO(), fn, retry.Attempts(5)) err2 := retry.Do(context.TODO(), fn, retry.Attempts(5))
if err2 != nil { if err2 != nil {
return err2 return err2
} }
} }
return nil return nil
} }
// UpdateVersion updates the version of the index meta, whenever the task is built once, the version will be updated once. func (mt *metaTable) ResetNodeID(buildID UniqueID) error {
func (mt *metaTable) UpdateVersion(indexBuildID UniqueID) error {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
log.Debug("IndexCoord metaTable update UpdateVersion", zap.Any("IndexBuildId", indexBuildID))
meta, ok := mt.indexBuildID2Meta[indexBuildID] log.Info("IndexCoord metaTable ResetNodeID", zap.Int64("buildID", buildID))
if !ok { updateFunc := func(m *Meta) error {
log.Warn("IndexCoord metaTable update UpdateVersion indexBuildID not exists", zap.Any("IndexBuildId", indexBuildID)) m.indexMeta.NodeID = 0
return fmt.Errorf("index not exists with ID = %d", indexBuildID) return mt.saveIndexMeta(m)
} }
//if meta.indexMeta.State != commonpb.IndexState_Unissued { if err := mt.updateMeta(buildID, updateFunc); err != nil {
// return fmt.Errorf("can not set lease key, index with ID = %d state is %d", indexBuildID, meta.indexMeta.State) log.Error("IndexCoord metaTable ResetNodeID fail", zap.Int64("buildID", buildID), zap.Error(err))
//} return err
meta.indexMeta.Version = meta.indexMeta.Version + 1
log.Debug("IndexCoord metaTable update UpdateVersion", zap.Any("IndexBuildId", indexBuildID),
zap.Any("Version", meta.indexMeta.Version))
err := mt.saveIndexMeta(&meta)
if err != nil {
fn := func() error {
m, err := mt.reloadMeta(meta.indexMeta.IndexBuildID)
if m == nil {
return err
}
m.indexMeta.Version = m.indexMeta.Version + 1
return mt.saveIndexMeta(m)
}
err2 := retry.Do(context.TODO(), fn, retry.Attempts(5))
return err2
} }
log.Info("reset index meta nodeID success", zap.Int64("buildID", buildID))
return nil return nil
} }
func (mt *metaTable) ResetMeta(buildID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()
log.Info("IndexCoord metaTable ResetMeta", zap.Int64("buildID", buildID))
updateFunc := func(m *Meta) error {
m.indexMeta.NodeID = 0
m.indexMeta.State = commonpb.IndexState_Unissued
return mt.saveIndexMeta(m)
}
if err := mt.updateMeta(buildID, updateFunc); err != nil {
log.Error("IndexCoord metaTable ResetMeta fail", zap.Int64("buildID", buildID), zap.Error(err))
return err
}
log.Info("reset index meta success", zap.Int64("buildID", buildID))
return nil
}
func (mt *metaTable) GetMeta(buildID UniqueID) (*Meta, bool) {
mt.lock.RLock()
defer mt.lock.RUnlock()
meta, ok := mt.indexBuildID2Meta[buildID]
if ok {
return &Meta{indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), etcdVersion: meta.etcdVersion}, ok
}
return nil, ok
}
func (mt *metaTable) canIndex(buildID int64) bool {
meta := mt.indexBuildID2Meta[buildID]
if meta.indexMeta.MarkDeleted {
log.Warn("Index has been deleted", zap.Int64("buildID", buildID))
return false
}
if meta.indexMeta.NodeID != 0 {
log.Error("IndexCoord metaTable BuildIndex, but indexMeta's NodeID is not zero",
zap.Int64("buildID", buildID), zap.Int64("nodeID", meta.indexMeta.NodeID))
return false
}
if meta.indexMeta.State != commonpb.IndexState_Unissued {
log.Error("IndexCoord metaTable BuildIndex, but indexMeta's state is not unissued",
zap.Int64("buildID", buildID), zap.String("state", meta.indexMeta.State.String()))
return false
}
return true
}
// UpdateVersion updates the version and nodeID of the index meta, whenever the task is built once, the version will be updated once.
func (mt *metaTable) UpdateVersion(indexBuildID UniqueID, nodeID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()
log.Info("IndexCoord metaTable UpdateVersion", zap.Int64("IndexBuildId", indexBuildID))
updateFunc := func(m *Meta) error {
if !mt.canIndex(indexBuildID) {
return fmt.Errorf("it's no necessary to build index with ID = %d", indexBuildID)
}
m.indexMeta.NodeID = nodeID
m.indexMeta.IndexVersion++
return mt.saveIndexMeta(m)
}
if err := mt.updateMeta(indexBuildID, updateFunc); err != nil {
log.Error("IndexCoord metaTable UpdateVersion fail", zap.Int64("buildID", indexBuildID), zap.Error(err))
return err
}
log.Info("IndexCoord metaTable UpdateVersion success", zap.Int64("IndexBuildId", indexBuildID),
zap.Int64("nodeID", nodeID))
return nil
}
// BuildIndex set the index state to be InProgress. It means IndexNode is building the index.
func (mt *metaTable) BuildIndex(indexBuildID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()
log.Debug("IndexCoord metaTable BuildIndex")
updateFunc := func(m *Meta) error {
if m.indexMeta.MarkDeleted {
log.Warn("index has been marked deleted, no need to build index", zap.Int64("indexBuildID", indexBuildID))
return nil
}
if m.indexMeta.State == commonpb.IndexState_Finished || m.indexMeta.State == commonpb.IndexState_Failed {
log.Warn("index has been finished, no need to set InProgress state", zap.Int64("indexBuildID", indexBuildID),
zap.String("state", m.indexMeta.State.String()))
return nil
}
m.indexMeta.State = commonpb.IndexState_InProgress
err := mt.saveIndexMeta(m)
if err != nil {
log.Error("IndexCoord metaTable BuildIndex fail", zap.Int64("buildID", indexBuildID), zap.Error(err))
return err
}
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.UnissuedIndexTaskLabel).Dec()
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.InProgressIndexTaskLabel).Inc()
log.Info("IndexCoord metaTable BuildIndex success", zap.Int64("buildID", indexBuildID),
zap.String("state", m.indexMeta.State.String()))
return nil
}
return mt.updateMeta(indexBuildID, updateFunc)
}
func (mt *metaTable) GetMetasByNodeID(nodeID UniqueID) []Meta {
mt.lock.RLock()
defer mt.lock.RUnlock()
metas := make([]Meta, 0)
for _, meta := range mt.indexBuildID2Meta {
if meta.indexMeta.MarkDeleted {
continue
}
if nodeID == meta.indexMeta.NodeID {
metas = append(metas, Meta{indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), etcdVersion: meta.etcdVersion})
}
}
return metas
}
// MarkIndexAsDeleted will mark the corresponding index as deleted, and recycleUnusedIndexFiles will recycle these tasks. // MarkIndexAsDeleted will mark the corresponding index as deleted, and recycleUnusedIndexFiles will recycle these tasks.
func (mt *metaTable) MarkIndexAsDeleted(indexID UniqueID) error { func (mt *metaTable) MarkIndexAsDeleted(indexID UniqueID) ([]UniqueID, error) {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
log.Debug("IndexCoord metaTable MarkIndexAsDeleted ", zap.Int64("indexID", indexID)) log.Info("IndexCoord metaTable MarkIndexAsDeleted ", zap.Int64("indexID", indexID))
deletedBuildIDs := make([]UniqueID, 0)
updateFunc := func(m *Meta) error {
m.indexMeta.MarkDeleted = true
log.Debug("IndexCoord metaTable MarkIndexAsDeleted ", zap.Int64("indexID", indexID),
zap.Int64("buildID", m.indexMeta.IndexBuildID))
return mt.saveIndexMeta(m)
}
for buildID, meta := range mt.indexBuildID2Meta { for buildID, meta := range mt.indexBuildID2Meta {
if meta.indexMeta.Req.IndexID == indexID && !meta.indexMeta.MarkDeleted { if meta.indexMeta.Req.IndexID == indexID {
meta.indexMeta.MarkDeleted = true deletedBuildIDs = append(deletedBuildIDs, buildID)
log.Debug("IndexCoord metaTable MarkIndexAsDeleted ", zap.Int64("indexID", indexID), if meta.indexMeta.MarkDeleted {
zap.Int64("buildID", buildID)) continue
// marshal inside }
/* #nosec G601 */ if err := mt.updateMeta(buildID, updateFunc); err != nil {
if err := mt.saveIndexMeta(&meta); err != nil { log.Error("IndexCoord metaTable mark index as deleted failed", zap.Int64("buildID", buildID))
log.Error("IndexCoord metaTable MarkIndexAsDeleted saveIndexMeta failed", zap.Error(err)) return nil, err
fn := func() error {
m, err := mt.reloadMeta(meta.indexMeta.IndexBuildID)
if m == nil {
return err
}
m.indexMeta.MarkDeleted = true
return mt.saveIndexMeta(m)
}
err2 := retry.Do(context.TODO(), fn, retry.Attempts(5))
if err2 != nil {
return err2
}
} }
} }
} }
log.Debug("IndexCoord metaTable MarkIndexAsDeleted success", zap.Int64("indexID", indexID)) log.Info("IndexCoord metaTable MarkIndexAsDeleted success", zap.Int64("indexID", indexID))
return nil return deletedBuildIDs, nil
} }
func (mt *metaTable) MarkIndexAsDeletedByBuildIDs(buildIDs []UniqueID) error { func (mt *metaTable) MarkIndexAsDeletedByBuildIDs(buildIDs []UniqueID) error {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
log.Debug("IndexCoord metaTable MarkIndexAsDeletedByBuildIDs ", zap.Int64s("buildIDs", buildIDs)) log.Debug("IndexCoord metaTable MarkIndexAsDeletedByBuildIDs", zap.Int64s("buildIDs", buildIDs))
updateFunc := func(m *Meta) error {
if m.indexMeta.MarkDeleted {
return nil
}
m.indexMeta.MarkDeleted = true
log.Debug("IndexCoord metaTable MarkIndexAsDeletedByBuildIDs ",
zap.Int64("buildID", m.indexMeta.IndexBuildID))
return mt.saveIndexMeta(m)
}
for _, buildID := range buildIDs { for _, buildID := range buildIDs {
if meta, ok := mt.indexBuildID2Meta[buildID]; ok { if _, ok := mt.indexBuildID2Meta[buildID]; !ok {
clonedMeta := &Meta{ continue
indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), }
indexVersion: meta.indexVersion, if err := mt.updateMeta(buildID, updateFunc); err != nil {
} log.Error("IndexCoord metaTable MarkIndexAsDeletedByBuildIDs fail", zap.Int64("buildID", buildID),
clonedMeta.indexMeta.MarkDeleted = true zap.Error(err))
// marshal inside return err
/* #nosec G601 */
if err := mt.saveIndexMeta(clonedMeta); err != nil {
log.Error("IndexCoord metaTable MarkIndexAsDeleted saveIndexMeta failed", zap.Error(err))
fn := func() error {
m, err := mt.reloadMeta(meta.indexMeta.IndexBuildID)
if m == nil {
return err
}
m.indexMeta.MarkDeleted = true
return mt.saveIndexMeta(m)
}
err2 := retry.Do(context.TODO(), fn, retry.Attempts(5))
if err2 != nil {
return err2
}
}
} }
} }
log.Debug("IndexCoord metaTable MarkIndexAsDeletedByBuildIDs success", zap.Int64s("buildIDs", buildIDs)) log.Info("IndexCoord metaTable MarkIndexAsDeletedByBuildIDs success", zap.Int64s("buildIDs", buildIDs))
return nil return nil
} }
@ -384,118 +473,44 @@ func (mt *metaTable) GetIndexFilePathInfo(indexBuildID UniqueID) (*indexpb.Index
} }
// DeleteIndex delete the index meta from meta table. // DeleteIndex delete the index meta from meta table.
func (mt *metaTable) DeleteIndex(indexBuildID UniqueID) { func (mt *metaTable) DeleteIndex(indexBuildID UniqueID) error {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
delete(mt.indexBuildID2Meta, indexBuildID)
key := path.Join(indexFilePrefix, strconv.FormatInt(indexBuildID, 10)) key := path.Join(indexFilePrefix, strconv.FormatInt(indexBuildID, 10))
if err := mt.client.Remove(key); err != nil { if err := mt.client.Remove(key); err != nil {
log.Error("IndexCoord delete index meta from etcd failed", zap.Error(err)) log.Error("IndexCoord delete index meta from etcd failed", zap.Error(err))
return err
} }
delete(mt.indexBuildID2Meta, indexBuildID)
log.Debug("IndexCoord delete index meta successfully", zap.Int64("indexBuildID", indexBuildID)) log.Debug("IndexCoord delete index meta successfully", zap.Int64("indexBuildID", indexBuildID))
}
// UpdateRecycleState update the recycle state corresponding the indexBuildID,
// when the recycle state is true, means the index files has been recycled with lower version.
func (mt *metaTable) UpdateRecycleState(indexBuildID UniqueID) error {
mt.lock.Lock()
defer mt.lock.Unlock()
meta, ok := mt.indexBuildID2Meta[indexBuildID]
log.Debug("IndexCoord metaTable UpdateRecycleState", zap.Any("indexBuildID", indexBuildID),
zap.Any("exists", ok))
if !ok {
return fmt.Errorf("index not exists with ID = %d", indexBuildID)
}
if meta.indexMeta.Recycled {
return nil
}
meta.indexMeta.Recycled = true
if err := mt.saveIndexMeta(&meta); err != nil {
fn := func() error {
m, err := mt.reloadMeta(meta.indexMeta.IndexBuildID)
if m == nil {
return err
}
m.indexMeta.Recycled = true
return mt.saveIndexMeta(m)
}
err2 := retry.Do(context.TODO(), fn, retry.Attempts(5))
if err2 != nil {
meta.indexMeta.Recycled = false
log.Error("IndexCoord metaTable UpdateRecycleState failed", zap.Error(err2))
return err2
}
}
return nil return nil
} }
// GetUnusedIndexFiles get the index files with lower version or corresponding the indexBuildIDs which has been deleted. func (mt *metaTable) GetBuildID2IndexFiles() map[UniqueID][]string {
func (mt *metaTable) GetUnusedIndexFiles(limit int) []Meta {
mt.lock.Lock()
defer mt.lock.Unlock()
var metas []Meta
for _, meta := range mt.indexBuildID2Meta {
if meta.indexMeta.State == commonpb.IndexState_Finished && (meta.indexMeta.MarkDeleted || !meta.indexMeta.Recycled) {
metas = append(metas, Meta{indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), indexVersion: meta.indexVersion})
}
if meta.indexMeta.State == commonpb.IndexState_Unissued && meta.indexMeta.MarkDeleted {
metas = append(metas, Meta{indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), indexVersion: meta.indexVersion})
}
if len(metas) >= limit {
return metas
}
}
return metas
}
func sortMetaPolicy(metas []Meta) []Meta {
// the larger the segment, the higher the priority
sort.Slice(metas, func(i, j int) bool {
return metas[i].indexMeta.Version < metas[j].indexMeta.Version ||
(metas[i].indexMeta.Version == metas[j].indexMeta.Version &&
metas[i].indexMeta.Req.NumRows > metas[j].indexMeta.Req.NumRows)
})
return metas
}
// GetUnassignedTasks get the unassigned tasks.
func (mt *metaTable) GetUnassignedTasks(onlineNodeIDs []int64) []Meta {
mt.lock.RLock() mt.lock.RLock()
defer mt.lock.RUnlock() defer mt.lock.RUnlock()
var metas []Meta
buildID2IndexFiles := make(map[UniqueID][]string)
for buildID, meta := range mt.indexBuildID2Meta {
buildID2IndexFiles[buildID] = append(buildID2IndexFiles[buildID], meta.indexMeta.IndexFilePaths...)
}
return buildID2IndexFiles
}
func (mt *metaTable) GetDeletedMetas() []*indexpb.IndexMeta {
mt.lock.RLock()
defer mt.lock.RUnlock()
var metas []*indexpb.IndexMeta
for _, meta := range mt.indexBuildID2Meta { for _, meta := range mt.indexBuildID2Meta {
if meta.indexMeta.MarkDeleted { if meta.indexMeta.MarkDeleted {
continue metas = append(metas, proto.Clone(meta.indexMeta).(*indexpb.IndexMeta))
}
if meta.indexMeta.State == commonpb.IndexState_Unissued {
metas = append(metas, Meta{indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), indexVersion: meta.indexVersion})
continue
}
if meta.indexMeta.State == commonpb.IndexState_Finished || meta.indexMeta.State == commonpb.IndexState_Failed {
continue
}
alive := false
for _, serverID := range onlineNodeIDs {
if meta.indexMeta.NodeID == serverID {
alive = true
break
}
}
if !alive {
log.Info("Reassign because node no longer alive", zap.Any("onlineID", onlineNodeIDs), zap.Int64("nodeID", meta.indexMeta.NodeID))
metas = append(metas, Meta{indexMeta: proto.Clone(meta.indexMeta).(*indexpb.IndexMeta), indexVersion: meta.indexVersion})
} }
} }
return sortMetaPolicy(metas) return metas
} }
// HasSameReq determine whether there are same indexing tasks. // HasSameReq determine whether there are same indexing tasks.
@ -560,70 +575,55 @@ func (mt *metaTable) HasSameReq(req *indexpb.BuildIndexRequest) (bool, UniqueID)
if notEq { if notEq {
continue continue
} }
if meta.indexMeta.MarkDeleted {
continue
}
return true, meta.indexMeta.IndexBuildID return true, meta.indexMeta.IndexBuildID
} }
return false, 0 return false, 0
} }
// LoadMetaFromETCD load the meta of specified indexBuildID from ETCD. // NeedUpdateMeta update the meta of specified indexBuildID.
// If the version of meta in memory is greater equal to the version in ETCD, no need to reload. // If the version of meta in memory is greater equal to the version in put event, no need to update.
func (mt *metaTable) LoadMetaFromETCD(indexBuildID int64, indexVersion int64) bool { func (mt *metaTable) NeedUpdateMeta(m *Meta) bool {
mt.lock.Lock() mt.lock.Lock()
defer mt.lock.Unlock() defer mt.lock.Unlock()
meta, ok := mt.indexBuildID2Meta[indexBuildID]
log.Debug("IndexCoord metaTable LoadMetaFromETCD", zap.Int64("indexBuildID", indexBuildID),
zap.Int64("indexVersion", indexVersion), zap.Bool("ok", ok))
if ok {
log.Debug("IndexCoord metaTable LoadMetaFromETCD",
zap.Int64("meta.indexVersion", meta.indexVersion),
zap.Int64("indexVersion", indexVersion))
if meta.indexVersion >= indexVersion { meta, ok := mt.indexBuildID2Meta[m.indexMeta.IndexBuildID]
return false
}
} else {
log.Error("Index not exist", zap.Int64("IndexBuildID", indexBuildID))
return false
}
m, err := mt.reloadMeta(indexBuildID)
if m == nil {
log.Error("IndexCoord metaTable reloadMeta failed", zap.Error(err))
return false
}
mt.indexBuildID2Meta[indexBuildID] = *m
log.Debug("IndexCoord LoadMetaFromETCD success", zap.Any("IndexMeta", m))
return true
}
// GetNodeTaskStats get task stats of IndexNode.
func (mt *metaTable) GetNodeTaskStats() map[UniqueID]int {
mt.lock.RLock()
defer mt.lock.RUnlock()
log.Debug("IndexCoord MetaTable GetPriorityForNodeID")
nodePriority := make(map[UniqueID]int)
for _, meta := range mt.indexBuildID2Meta {
if meta.indexMeta.State == commonpb.IndexState_InProgress {
nodePriority[meta.indexMeta.NodeID]++
}
}
return nodePriority
}
// GetIndexMetaByIndexBuildID get the index meta of the specified indexBuildID.
func (mt *metaTable) GetIndexMetaByIndexBuildID(indexBuildID UniqueID) *indexpb.IndexMeta {
mt.lock.RLock()
defer mt.lock.RUnlock()
log.Debug("IndexCoord MetaTable GetIndexMeta", zap.Int64("IndexBuildID", indexBuildID))
meta, ok := mt.indexBuildID2Meta[indexBuildID]
if !ok { if !ok {
log.Error("IndexCoord MetaTable GetIndexMeta not exist", zap.Int64("IndexBuildID", indexBuildID)) log.Warn("index is not exist, Might have been cleaned up meta", zap.Int64("buildID", m.indexMeta.IndexBuildID))
return nil return false
} }
return proto.Clone(meta.indexMeta).(*indexpb.IndexMeta) log.Info("IndexCoord metaTable NeedUpdateMeta", zap.Int64("indexBuildID", m.indexMeta.IndexBuildID),
zap.Int64("meta.revision", meta.etcdVersion), zap.Int64("update revision", m.etcdVersion))
if meta.etcdVersion < m.etcdVersion {
mt.indexBuildID2Meta[m.indexMeta.IndexBuildID] = m
return true
}
return false
}
func (mt *metaTable) CanBeRecycledIndexFiles(buildID UniqueID) bool {
mt.lock.RLock()
defer mt.lock.RUnlock()
meta, ok := mt.indexBuildID2Meta[buildID]
if !ok {
log.Debug("index meta not exist, can be recycled", zap.Int64("buildID", buildID))
return true
}
if meta.indexMeta.MarkDeleted {
log.Debug("index has been deleted, can be recycled", zap.Int64("buildID", buildID))
return true
}
if meta.indexMeta.State == commonpb.IndexState_Finished || meta.indexMeta.State == commonpb.IndexState_Failed {
log.Debug("index has been finished, can be recycled", zap.Int64("buildID", buildID),
zap.String("state", meta.indexMeta.State.String()))
return true
}
log.Debug("index meta can not be recycled", zap.Int64("buildID", buildID),
zap.Bool("deleted", meta.indexMeta.MarkDeleted), zap.String("state", meta.indexMeta.String()))
return false
} }

File diff suppressed because it is too large Load Diff

View File

@ -19,7 +19,9 @@ package indexcoord
import ( import (
"context" "context"
"sync" "sync"
"time"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/metrics"
@ -29,7 +31,6 @@ import (
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
) )
// NodeManager is used by IndexCoord to manage the client of IndexNode. // NodeManager is used by IndexCoord to manage the client of IndexNode.
@ -81,6 +82,7 @@ func (nm *NodeManager) RemoveNode(nodeID UniqueID) {
// AddNode adds the client of IndexNode. // AddNode adds the client of IndexNode.
func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error { func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error {
log.Debug("IndexCoord addNode", zap.Any("nodeID", nodeID), zap.Any("node address", address)) log.Debug("IndexCoord addNode", zap.Any("nodeID", nodeID), zap.Any("node address", address))
if nm.pq.CheckExist(nodeID) { if nm.pq.CheckExist(nodeID) {
log.Warn("IndexCoord", zap.Any("Node client already exist with ID:", nodeID)) log.Warn("IndexCoord", zap.Any("Node client already exist with ID:", nodeID))
@ -102,93 +104,31 @@ func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error {
} }
// PeekClient peeks the client with the least load. // PeekClient peeks the client with the least load.
func (nm *NodeManager) PeekClient(meta Meta) (UniqueID, types.IndexNode) { func (nm *NodeManager) PeekClient(meta *Meta) (UniqueID, types.IndexNode) {
log.Debug("IndexCoord NodeManager PeekClient")
dataSize, err := estimateIndexSizeByReq(meta.indexMeta.Req)
if err != nil {
log.Warn(err.Error())
return UniqueID(-1), nil
}
log.Debug("IndexCoord peek IndexNode client from pq", zap.Uint64("data size", dataSize))
nodeID := nm.pq.Peek(dataSize*indexSizeFactor, meta.indexMeta.Req.IndexParams, meta.indexMeta.Req.TypeParams)
if nodeID == -1 {
log.Error("there is no indexnode online")
return nodeID, nil
}
if nodeID == 0 {
log.Error("No IndexNode available", zap.Uint64("data size", dataSize),
zap.Uint64("IndexNode must have memory size", dataSize*indexSizeFactor))
return nodeID, nil
}
nm.lock.Lock()
defer nm.lock.Unlock()
client, ok := nm.nodeClients[nodeID]
if !ok {
log.Error("IndexCoord NodeManager PeekClient", zap.Int64("There is no IndexNode client corresponding to NodeID", nodeID))
return nodeID, nil
}
log.Debug("IndexCoord NodeManager PeekClient ", zap.Int64("node", nodeID), zap.Uint64("data size", dataSize))
return nodeID, client
}
// ListNode lists all IndexNodes in node manager.
func (nm *NodeManager) ListNode() []UniqueID {
//nm.lock.Lock()
//defer nm.lock.Unlock()
var clientIDs []UniqueID
nm.lock.RLock() nm.lock.RLock()
for id := range nm.nodeClients { defer nm.lock.RUnlock()
clientIDs = append(clientIDs, id)
if len(nm.nodeClients) == 0 {
log.Error("there is no IndexNode online")
return -1, nil
} }
for nodeID, client := range nm.nodeClients {
nm.lock.RUnlock() resp, err := client.GetTaskSlots(nm.ctx, &indexpb.GetTaskSlotsRequest{})
var wg sync.WaitGroup if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
for _, id := range clientIDs { continue
memory := nm.pq.GetMemory(id) }
if memory == 0 { if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Debug("IndexCoord get IndexNode metrics info", zap.Int64("nodeID", id)) log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) zap.String("reason", resp.Status.Reason))
if err != nil { continue
log.Error("create metrics request failed", zap.Error(err)) }
continue if resp.Slots > 0 {
} return nodeID, client
nm.lock.RLock()
client, ok := nm.nodeClients[id]
if !ok {
nm.lock.RUnlock()
log.Debug("NodeManager ListNode find client not exist")
continue
}
nm.lock.RUnlock()
wg.Add(1)
go func(group *sync.WaitGroup, id UniqueID) {
defer group.Done()
ctx, cancel := context.WithTimeout(nm.ctx, time.Second*5)
defer cancel()
metrics, err := client.GetMetrics(ctx, req)
if err != nil {
log.Error("get IndexNode metrics failed", zap.Error(err))
return
}
infos := &metricsinfo.IndexNodeInfos{}
err = metricsinfo.UnmarshalComponentInfos(metrics.Response, infos)
if err != nil {
log.Error("get IndexNode metrics info failed", zap.Error(err))
return
}
log.Debug("IndexCoord get IndexNode's metrics success", zap.Int64("nodeID", id),
zap.Int("CPUCoreCount", infos.HardwareInfos.CPUCoreCount), zap.Float64("CPUCoreUsage", infos.HardwareInfos.CPUCoreUsage),
zap.Uint64("Memory", infos.HardwareInfos.Memory), zap.Uint64("MemoryUsage", infos.HardwareInfos.MemoryUsage))
nm.pq.SetMemory(id, infos.HardwareInfos.Memory)
}(&wg, id)
} }
} }
wg.Wait()
return clientIDs return 0, nil
} }
// indexNodeGetMetricsResponse record the metrics information of IndexNode. // indexNodeGetMetricsResponse record the metrics information of IndexNode.

View File

@ -28,7 +28,7 @@ import (
func TestNodeManager_PeekClient(t *testing.T) { func TestNodeManager_PeekClient(t *testing.T) {
nm := NewNodeManager(context.Background()) nm := NewNodeManager(context.Background())
meta := Meta{ meta := &Meta{
indexMeta: &indexpb.IndexMeta{ indexMeta: &indexpb.IndexMeta{
Req: &indexpb.BuildIndexRequest{ Req: &indexpb.BuildIndexRequest{
DataPaths: []string{"PeekClient-1", "PeekClient-2"}, DataPaths: []string{"PeekClient-1", "PeekClient-2"},

View File

@ -0,0 +1,32 @@
package indexcoord
type indexTaskState int32
const (
// when we receive a index task
indexTaskInit indexTaskState = iota
// we've sent index task to scheduler, and wait for building index.
indexTaskInProgress
// task done, wait to be cleaned
indexTaskDone
// index task need to retry.
indexTaskRetry
// task has been deleted.
indexTaskDeleted
)
var TaskStateNames = map[indexTaskState]string{
0: "Init",
1: "InProgress",
2: "Done",
3: "Retry",
4: "Deleted",
}
func (x indexTaskState) String() string {
ret, ok := TaskStateNames[x]
if !ok {
return "None"
}
return ret
}

View File

@ -19,6 +19,7 @@ package indexcoord
import ( import (
"errors" "errors"
"strconv" "strconv"
"strings"
"github.com/milvus-io/milvus/internal/util/funcutil" "github.com/milvus-io/milvus/internal/util/funcutil"
@ -83,3 +84,11 @@ func estimateIndexSizeByReq(req *indexpb.BuildIndexRequest) (uint64, error) {
} }
return estimateScalarIndexSize(req) return estimateScalarIndexSize(req)
} }
func parseBuildIDFromFilePath(key string) (UniqueID, error) {
ss := strings.Split(key, "/")
if strings.HasSuffix(key, "/") {
return strconv.ParseInt(ss[len(ss)-2], 10, 64)
}
return strconv.ParseInt(ss[len(ss)-1], 10, 64)
}

View File

@ -203,3 +203,14 @@ func Test_estimateIndexSizeByReq(t *testing.T) {
}) })
} }
} }
func Test_parseKey(t *testing.T) {
key := "test-ListObjects/1/"
buildID, err := parseBuildIDFromFilePath(key)
assert.Nil(t, err)
assert.Equal(t, int64(1), buildID)
key2 := "test-ListObjects/key1/"
_, err2 := parseBuildIDFromFilePath(key2)
assert.Error(t, err2)
}

View File

@ -21,6 +21,10 @@ import (
"fmt" "fmt"
) )
var (
ErrNoSuchKey = errors.New("NoSuchKey")
)
// msgIndexNodeIsUnhealthy return a message tha IndexNode is not healthy. // msgIndexNodeIsUnhealthy return a message tha IndexNode is not healthy.
func msgIndexNodeIsUnhealthy(nodeID UniqueID) string { func msgIndexNodeIsUnhealthy(nodeID UniqueID) string {
return fmt.Sprintf("index node %d is not ready", nodeID) return fmt.Sprintf("index node %d is not ready", nodeID)

View File

@ -255,8 +255,8 @@ func (i *IndexNode) UpdateStateCode(code internalpb.StateCode) {
} }
// SetEtcdClient assigns parameter client to its member etcdCli // SetEtcdClient assigns parameter client to its member etcdCli
func (node *IndexNode) SetEtcdClient(client *clientv3.Client) { func (i *IndexNode) SetEtcdClient(client *clientv3.Client) {
node.etcdCli = client i.etcdCli = client
} }
func (i *IndexNode) isHealthy() bool { func (i *IndexNode) isHealthy() bool {
@ -318,6 +318,29 @@ func (i *IndexNode) CreateIndex(ctx context.Context, request *indexpb.CreateInde
return ret, nil return ret, nil
} }
// GetTaskSlots gets how many task the IndexNode can still perform.
func (i *IndexNode) GetTaskSlots(ctx context.Context, req *indexpb.GetTaskSlotsRequest) (*indexpb.GetTaskSlotsResponse, error) {
if i.stateCode.Load().(internalpb.StateCode) != internalpb.StateCode_Healthy {
return &indexpb.GetTaskSlotsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "state code is not healthy",
},
}, nil
}
log.Info("IndexNode GetTaskSlots received")
ret := &indexpb.GetTaskSlotsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}
ret.Slots = int64(i.sched.GetTaskSlots())
log.Info("IndexNode GetTaskSlots success", zap.Int64("slots", ret.Slots))
return ret, nil
}
// GetComponentStates gets the component states of IndexNode. // GetComponentStates gets the component states of IndexNode.
func (i *IndexNode) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { func (i *IndexNode) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
log.Debug("get IndexNode components states ...") log.Debug("get IndexNode components states ...")

View File

@ -60,7 +60,7 @@ func (inm *Mock) Init() error {
return errors.New("IndexNode init failed") return errors.New("IndexNode init failed")
} }
inm.ctx, inm.cancel = context.WithCancel(context.Background()) inm.ctx, inm.cancel = context.WithCancel(context.Background())
inm.buildIndex = make(chan *indexpb.CreateIndexRequest) inm.buildIndex = make(chan *indexpb.CreateIndexRequest, 10)
return nil return nil
} }
@ -117,12 +117,11 @@ func (inm *Mock) buildIndexTask() {
return err return err
} }
indexMeta.IndexFilePaths = []string{"IndexFilePath-1", "IndexFilePath-2"} indexMeta.IndexFilePaths = []string{"IndexFilePath-1", "IndexFilePath-2"}
indexMeta.State = commonpb.IndexState_Failed indexMeta.State = commonpb.IndexState_Finished
metaData, err := proto.Marshal(&indexMeta) metaData, err := proto.Marshal(&indexMeta)
if err != nil { if err != nil {
return err return err
} }
success, err := inm.etcdKV.CompareVersionAndSwap(req.MetaPath, versions[0], string(metaData)) success, err := inm.etcdKV.CompareVersionAndSwap(req.MetaPath, versions[0], string(metaData))
if err != nil { if err != nil {
// TODO, we don't need to reload if it is just etcd error // TODO, we don't need to reload if it is just etcd error
@ -132,32 +131,6 @@ func (inm *Mock) buildIndexTask() {
if !success { if !success {
return fmt.Errorf("failed to save index meta in etcd, buildId: %d, source version: %d", req.IndexBuildID, versions[0]) return fmt.Errorf("failed to save index meta in etcd, buildId: %d, source version: %d", req.IndexBuildID, versions[0])
} }
indexMeta2 := indexpb.IndexMeta{}
_, values2, versions2, err := inm.etcdKV.LoadWithPrefix2(req.MetaPath)
if err != nil {
return err
}
err = proto.Unmarshal([]byte(values2[0]), &indexMeta2)
if err != nil {
return err
}
indexMeta2.Version = indexMeta.Version + 1
indexMeta2.IndexFilePaths = []string{"IndexFilePath-1", "IndexFilePath-2"}
indexMeta2.State = commonpb.IndexState_Finished
metaData2, err := proto.Marshal(&indexMeta2)
if err != nil {
return err
}
success, err = inm.etcdKV.CompareVersionAndSwap(req.MetaPath, versions2[0], string(metaData2))
if err != nil {
// TODO, we don't need to reload if it is just etcd error
log.Warn("failed to compare and swap in etcd", zap.Int64("buildID", req.IndexBuildID), zap.Error(err))
return err
}
if !success {
return fmt.Errorf("failed to save index meta in etcd, buildId: %d, source version: %d", req.IndexBuildID, versions[0])
}
return nil return nil
} }
err := retry.Do(context.Background(), saveIndexMeta, retry.Attempts(3)) err := retry.Do(context.Background(), saveIndexMeta, retry.Attempts(3))
@ -343,6 +316,35 @@ func (inm *Mock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
}, nil }, nil
} }
func (inm *Mock) GetTaskSlots(ctx context.Context, req *indexpb.GetTaskSlotsRequest) (*indexpb.GetTaskSlotsResponse, error) {
if inm.Err {
return &indexpb.GetTaskSlotsResponse{
Slots: 0,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "IndexNode mock err",
},
}, errors.New("IndexNode GetMetrics failed")
}
if inm.Failure {
return &indexpb.GetTaskSlotsResponse{
Slots: 0,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "IndexNode mock fail",
},
}, nil
}
return &indexpb.GetTaskSlotsResponse{
Slots: 1,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
}
func getMockSystemInfoMetrics( func getMockSystemInfoMetrics(
ctx context.Context, ctx context.Context,
req *milvuspb.GetMetricsRequest, req *milvuspb.GetMetricsRequest,

View File

@ -176,7 +176,7 @@ func TestIndexNodeMockFiled(t *testing.T) {
indexMeta := &indexpb.IndexMeta{ indexMeta := &indexpb.IndexMeta{
IndexBuildID: 10, IndexBuildID: 10,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
Version: 0, IndexVersion: 0,
} }
value, err := proto.Marshal(indexMeta) value, err := proto.Marshal(indexMeta)

View File

@ -138,7 +138,7 @@ func TestIndexNode(t *testing.T) {
indexMeta := &indexpb.IndexMeta{ indexMeta := &indexpb.IndexMeta{
IndexBuildID: indexBuildID1, IndexBuildID: indexBuildID1,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
Version: 1, IndexVersion: 1,
} }
value, err := proto.Marshal(indexMeta) value, err := proto.Marshal(indexMeta)
@ -253,7 +253,7 @@ func TestIndexNode(t *testing.T) {
indexMeta := &indexpb.IndexMeta{ indexMeta := &indexpb.IndexMeta{
IndexBuildID: indexBuildID2, IndexBuildID: indexBuildID2,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
Version: 1, IndexVersion: 1,
} }
value, err := proto.Marshal(indexMeta) value, err := proto.Marshal(indexMeta)
@ -365,7 +365,7 @@ func TestIndexNode(t *testing.T) {
indexMeta := &indexpb.IndexMeta{ indexMeta := &indexpb.IndexMeta{
IndexBuildID: indexBuildID1, IndexBuildID: indexBuildID1,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
Version: 1, IndexVersion: 1,
MarkDeleted: true, MarkDeleted: true,
} }
@ -412,7 +412,7 @@ func TestIndexNode(t *testing.T) {
err = proto.Unmarshal([]byte(strValue), &indexMetaTmp) err = proto.Unmarshal([]byte(strValue), &indexMetaTmp)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, true, indexMetaTmp.MarkDeleted) assert.Equal(t, true, indexMetaTmp.MarkDeleted)
assert.Equal(t, int64(1), indexMetaTmp.Version) assert.Equal(t, int64(1), indexMetaTmp.IndexVersion)
//for indexMetaTmp.State != commonpb.IndexState_Finished { //for indexMetaTmp.State != commonpb.IndexState_Finished {
// time.Sleep(100 * time.Millisecond) // time.Sleep(100 * time.Millisecond)
// strValue, err := in.etcdKV.Load(metaPath3) // strValue, err := in.etcdKV.Load(metaPath3)
@ -553,7 +553,7 @@ func TestCreateIndexFailed(t *testing.T) {
indexMeta := &indexpb.IndexMeta{ indexMeta := &indexpb.IndexMeta{
IndexBuildID: indexBuildID1, IndexBuildID: indexBuildID1,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
Version: 1, IndexVersion: 1,
} }
value, err := proto.Marshal(indexMeta) value, err := proto.Marshal(indexMeta)
@ -671,7 +671,7 @@ func TestCreateIndexFailed(t *testing.T) {
indexMeta2 := &indexpb.IndexMeta{ indexMeta2 := &indexpb.IndexMeta{
IndexBuildID: indexBuildID2, IndexBuildID: indexBuildID2,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
Version: 1, IndexVersion: 1,
} }
value2, err := proto.Marshal(indexMeta2) value2, err := proto.Marshal(indexMeta2)

View File

@ -200,10 +200,10 @@ func (it *IndexBuildTask) updateTaskState(indexMeta *indexpb.IndexMeta, err erro
if err != nil { if err != nil {
log.Warn("IndexNode IndexBuildTask internal err, mark the task as retry", zap.Int64("buildID", it.req.IndexBuildID), zap.Error(err)) log.Warn("IndexNode IndexBuildTask internal err, mark the task as retry", zap.Int64("buildID", it.req.IndexBuildID), zap.Error(err))
it.SetState(TaskStateRetry) it.SetState(TaskStateRetry)
} else if indexMeta.Version > it.req.Version || indexMeta.State == commonpb.IndexState_Finished { } else if indexMeta.IndexVersion > it.req.Version || indexMeta.State == commonpb.IndexState_Finished {
it.SetState(TaskStateAbandon) it.SetState(TaskStateAbandon)
} else if indexMeta.MarkDeleted { } else if indexMeta.MarkDeleted {
it.SetState(TaskStateDeleted) it.SetState(TaskStateAbandon)
} }
return it.GetState() return it.GetState()
} }
@ -216,7 +216,7 @@ func (it *IndexBuildTask) saveIndexMeta(ctx context.Context) error {
fn := func() error { fn := func() error {
indexMeta, version, err := it.loadIndexMeta(ctx) indexMeta, version, err := it.loadIndexMeta(ctx)
if err != nil { if err != nil {
log.Info("IndexNode IndexBuildTask saveIndexMeta fail to load index meta,", zap.Int64("build Id", indexMeta.IndexBuildID), zap.Error(err)) log.Error("IndexNode IndexBuildTask saveIndexMeta fail to load index meta,", zap.Int64("build Id", indexMeta.IndexBuildID), zap.Error(err))
return err return err
} }
taskState := it.updateTaskState(indexMeta, it.internalErr) taskState := it.updateTaskState(indexMeta, it.internalErr)
@ -224,11 +224,9 @@ func (it *IndexBuildTask) saveIndexMeta(ctx context.Context) error {
log.Warn("IndexNode IndexBuildTask saveIndexMeta success because task abandon", zap.String("TaskState", taskState.String()), log.Warn("IndexNode IndexBuildTask saveIndexMeta success because task abandon", zap.String("TaskState", taskState.String()),
zap.Int64("IndexBuildID", indexMeta.IndexBuildID)) zap.Int64("IndexBuildID", indexMeta.IndexBuildID))
return nil return nil
} else if taskState == TaskStateDeleted { }
log.Info("IndexNode IndexBuildTask saveIndexMeta", zap.String("TaskState", taskState.String()),
zap.Int64("IndexBuildID", indexMeta.IndexBuildID)) if taskState == TaskStateFailed {
indexMeta.State = commonpb.IndexState_Finished
} else if taskState == TaskStateFailed {
log.Error("IndexNode IndexBuildTask saveIndexMeta set indexMeta.state to IndexState_Failed", log.Error("IndexNode IndexBuildTask saveIndexMeta set indexMeta.state to IndexState_Failed",
zap.String("TaskState", taskState.String()), zap.String("TaskState", taskState.String()),
zap.Int64("IndexBuildID", indexMeta.IndexBuildID), zap.Error(it.err)) zap.Int64("IndexBuildID", indexMeta.IndexBuildID), zap.Error(it.err))
@ -348,6 +346,9 @@ func (it *IndexBuildTask) loadFieldData(ctx context.Context) (storage.FieldID, s
getValueByPath := func(path string) ([]byte, error) { getValueByPath := func(path string) ([]byte, error) {
data, err := it.cm.Read(path) data, err := it.cm.Read(path)
if err != nil { if err != nil {
if errors.Is(err, ErrNoSuchKey) {
return nil, ErrNoSuchKey
}
return nil, err return nil, err
} }
return data, nil return data, nil
@ -522,9 +523,9 @@ func (it *IndexBuildTask) saveIndex(ctx context.Context, blobs []*storage.Blob)
return err return err
} }
//log.Debug("IndexNode Unmarshal indexMeta success ", zap.Any("meta", indexMeta)) //log.Debug("IndexNode Unmarshal indexMeta success ", zap.Any("meta", indexMeta))
if indexMeta.Version > it.req.Version { if indexMeta.IndexVersion > it.req.Version {
log.Warn("IndexNode try saveIndexFile failed req.Version is low", zap.Any("req.Version", it.req.Version), log.Warn("IndexNode try saveIndexFile failed req.Version is low", zap.Any("req.Version", it.req.Version),
zap.Any("indexMeta.Version", indexMeta.Version)) zap.Any("indexMeta.Version", indexMeta.IndexVersion))
return errors.New("This task has been reassigned, check indexMeta.version and request ") return errors.New("This task has been reassigned, check indexMeta.version and request ")
} }
return it.cm.Write(savePath, blob.Value) return it.cm.Write(savePath, blob.Value)
@ -574,10 +575,15 @@ func (it *IndexBuildTask) Execute(ctx context.Context) error {
var blobs []*storage.Blob var blobs []*storage.Blob
blobs, err = it.buildIndex(ctx) blobs, err = it.buildIndex(ctx)
if err != nil { if err != nil {
it.SetState(TaskStateFailed) if errors.Is(err, ErrNoSuchKey) {
log.Error("IndexNode IndexBuildTask Execute buildIndex failed", it.SetState(TaskStateFailed)
zap.Int64("buildId", it.req.IndexBuildID), log.Error("IndexNode IndexBuildTask Execute buildIndex failed",
zap.Error(err)) zap.Int64("buildId", it.req.IndexBuildID), zap.Error(err))
return err
}
it.SetState(TaskStateRetry)
log.Error("IndexNode IndexBuildTask Execute buildIndex failed, need to retry",
zap.Int64("buildId", it.req.IndexBuildID), zap.Error(err))
return err return err
} }

View File

@ -43,6 +43,7 @@ type TaskQueue interface {
PopActiveTask(tID UniqueID) task PopActiveTask(tID UniqueID) task
Enqueue(t task) error Enqueue(t task) error
//tryToRemoveUselessIndexBuildTask(indexID UniqueID) []UniqueID //tryToRemoveUselessIndexBuildTask(indexID UniqueID) []UniqueID
GetTaskNum() int
} }
// BaseTaskQueue is a basic instance of TaskQueue. // BaseTaskQueue is a basic instance of TaskQueue.
@ -169,6 +170,18 @@ func (queue *BaseTaskQueue) Enqueue(t task) error {
return queue.addUnissuedTask(t) return queue.addUnissuedTask(t)
} }
func (queue *BaseTaskQueue) GetTaskNum() int {
queue.utLock.Lock()
utNum := queue.unissuedTasks.Len()
queue.utLock.Unlock()
queue.atLock.Lock()
atNum := len(queue.activeTasks)
queue.atLock.Unlock()
return utNum + atNum
}
// IndexBuildTaskQueue is a task queue used to store building index tasks. // IndexBuildTaskQueue is a task queue used to store building index tasks.
type IndexBuildTaskQueue struct { type IndexBuildTaskQueue struct {
BaseTaskQueue BaseTaskQueue
@ -206,7 +219,7 @@ func NewTaskScheduler(ctx context.Context,
cm: cm, cm: cm,
ctx: ctx1, ctx: ctx1,
cancel: cancel, cancel: cancel,
buildParallel: 1, // default value buildParallel: Params.IndexNodeCfg.BuildParallel,
} }
s.IndexBuildQueue = NewIndexBuildTaskQueue(s) s.IndexBuildQueue = NewIndexBuildTaskQueue(s)
@ -315,3 +328,7 @@ func (sched *TaskScheduler) Close() {
sched.cancel() sched.cancel()
sched.wg.Wait() sched.wg.Wait()
} }
func (sched *TaskScheduler) GetTaskSlots() int {
return sched.buildParallel - sched.IndexBuildQueue.GetTaskNum()
}

View File

@ -23,7 +23,6 @@ const (
TaskStateAbandon TaskState = 1 TaskStateAbandon TaskState = 1
TaskStateRetry TaskState = 2 TaskStateRetry TaskState = 2
TaskStateFailed TaskState = 3 TaskStateFailed TaskState = 3
TaskStateDeleted TaskState = 4
) )
var TaskStateNames = map[TaskState]string{ var TaskStateNames = map[TaskState]string{

View File

@ -24,14 +24,13 @@ import (
"strconv" "strconv"
"testing" "testing"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
@ -47,7 +46,7 @@ func TestIndexBuildTask_saveIndexMeta(t *testing.T) {
IndexBuildID: indexBuildID, IndexBuildID: indexBuildID,
State: commonpb.IndexState_InProgress, State: commonpb.IndexState_InProgress,
NodeID: 1, NodeID: 1,
Version: 1, IndexVersion: 1,
} }
metaPath := path.Join("indexes", strconv.FormatInt(indexMeta.IndexBuildID, 10)) metaPath := path.Join("indexes", strconv.FormatInt(indexMeta.IndexBuildID, 10))
metaValue, err := proto.Marshal(indexMeta) metaValue, err := proto.Marshal(indexMeta)
@ -77,3 +76,52 @@ func TestIndexBuildTask_saveIndexMeta(t *testing.T) {
err = etcdKV.Remove(metaPath) err = etcdKV.Remove(metaPath)
assert.NoError(t, err) assert.NoError(t, err)
} }
type mockChunkManager struct {
storage.ChunkManager
read func(key string) ([]byte, error)
}
func (mcm *mockChunkManager) Read(key string) ([]byte, error) {
return mcm.read(key)
}
func TestIndexBuildTask_Execute(t *testing.T) {
t.Run("task retry", func(t *testing.T) {
indexTask := &IndexBuildTask{
cm: &mockChunkManager{
read: func(key string) ([]byte, error) {
return nil, errors.New("error occurred")
},
},
req: &indexpb.CreateIndexRequest{
IndexBuildID: 1,
DataPaths: []string{"path1", "path2"},
},
}
err := indexTask.Execute(context.Background())
assert.Error(t, err)
assert.Equal(t, TaskStateRetry, indexTask.state)
})
t.Run("task failed", func(t *testing.T) {
indexTask := &IndexBuildTask{
cm: &mockChunkManager{
read: func(key string) ([]byte, error) {
return nil, ErrNoSuchKey
},
},
req: &indexpb.CreateIndexRequest{
IndexBuildID: 1,
DataPaths: []string{"path1", "path2"},
},
}
err := indexTask.Execute(context.Background())
assert.ErrorIs(t, err, ErrNoSuchKey)
assert.Equal(t, TaskStateFailed, indexTask.state)
})
}

View File

@ -28,6 +28,7 @@ service IndexNode {
rpc GetTimeTickChannel(internal.GetTimeTickChannelRequest) returns(milvus.StringResponse) {} rpc GetTimeTickChannel(internal.GetTimeTickChannelRequest) returns(milvus.StringResponse) {}
rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){} rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){}
rpc CreateIndex(CreateIndexRequest) returns (common.Status){} rpc CreateIndex(CreateIndexRequest) returns (common.Status){}
rpc GetTaskSlots(GetTaskSlotsRequest) returns (GetTaskSlotsResponse){}
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {} rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
@ -113,7 +114,7 @@ message IndexMeta {
repeated string index_file_paths = 5; repeated string index_file_paths = 5;
bool mark_deleted = 6; bool mark_deleted = 6;
int64 nodeID = 7; int64 nodeID = 7;
int64 version = 8; int64 index_version = 8;
bool recycled = 9; bool recycled = 9;
uint64 serialize_size = 10; uint64 serialize_size = 10;
} }
@ -125,3 +126,12 @@ message DropIndexRequest {
message RemoveIndexRequest { message RemoveIndexRequest {
repeated int64 buildIDs = 1; repeated int64 buildIDs = 1;
} }
message GetTaskSlotsRequest {
}
message GetTaskSlotsResponse {
common.Status status = 1;
int64 slots = 2;
}

View File

@ -689,7 +689,7 @@ type IndexMeta struct {
IndexFilePaths []string `protobuf:"bytes,5,rep,name=index_file_paths,json=indexFilePaths,proto3" json:"index_file_paths,omitempty"` IndexFilePaths []string `protobuf:"bytes,5,rep,name=index_file_paths,json=indexFilePaths,proto3" json:"index_file_paths,omitempty"`
MarkDeleted bool `protobuf:"varint,6,opt,name=mark_deleted,json=markDeleted,proto3" json:"mark_deleted,omitempty"` MarkDeleted bool `protobuf:"varint,6,opt,name=mark_deleted,json=markDeleted,proto3" json:"mark_deleted,omitempty"`
NodeID int64 `protobuf:"varint,7,opt,name=nodeID,proto3" json:"nodeID,omitempty"` NodeID int64 `protobuf:"varint,7,opt,name=nodeID,proto3" json:"nodeID,omitempty"`
Version int64 `protobuf:"varint,8,opt,name=version,proto3" json:"version,omitempty"` IndexVersion int64 `protobuf:"varint,8,opt,name=index_version,json=indexVersion,proto3" json:"index_version,omitempty"`
Recycled bool `protobuf:"varint,9,opt,name=recycled,proto3" json:"recycled,omitempty"` Recycled bool `protobuf:"varint,9,opt,name=recycled,proto3" json:"recycled,omitempty"`
SerializeSize uint64 `protobuf:"varint,10,opt,name=serialize_size,json=serializeSize,proto3" json:"serialize_size,omitempty"` SerializeSize uint64 `protobuf:"varint,10,opt,name=serialize_size,json=serializeSize,proto3" json:"serialize_size,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -771,9 +771,9 @@ func (m *IndexMeta) GetNodeID() int64 {
return 0 return 0
} }
func (m *IndexMeta) GetVersion() int64 { func (m *IndexMeta) GetIndexVersion() int64 {
if m != nil { if m != nil {
return m.Version return m.IndexVersion
} }
return 0 return 0
} }
@ -870,6 +870,84 @@ func (m *RemoveIndexRequest) GetBuildIDs() []int64 {
return nil return nil
} }
type GetTaskSlotsRequest struct {
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *GetTaskSlotsRequest) Reset() { *m = GetTaskSlotsRequest{} }
func (m *GetTaskSlotsRequest) String() string { return proto.CompactTextString(m) }
func (*GetTaskSlotsRequest) ProtoMessage() {}
func (*GetTaskSlotsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_f9e019eb3fda53c2, []int{14}
}
func (m *GetTaskSlotsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_GetTaskSlotsRequest.Unmarshal(m, b)
}
func (m *GetTaskSlotsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_GetTaskSlotsRequest.Marshal(b, m, deterministic)
}
func (m *GetTaskSlotsRequest) XXX_Merge(src proto.Message) {
xxx_messageInfo_GetTaskSlotsRequest.Merge(m, src)
}
func (m *GetTaskSlotsRequest) XXX_Size() int {
return xxx_messageInfo_GetTaskSlotsRequest.Size(m)
}
func (m *GetTaskSlotsRequest) XXX_DiscardUnknown() {
xxx_messageInfo_GetTaskSlotsRequest.DiscardUnknown(m)
}
var xxx_messageInfo_GetTaskSlotsRequest proto.InternalMessageInfo
type GetTaskSlotsResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
Slots int64 `protobuf:"varint,2,opt,name=slots,proto3" json:"slots,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
}
func (m *GetTaskSlotsResponse) Reset() { *m = GetTaskSlotsResponse{} }
func (m *GetTaskSlotsResponse) String() string { return proto.CompactTextString(m) }
func (*GetTaskSlotsResponse) ProtoMessage() {}
func (*GetTaskSlotsResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_f9e019eb3fda53c2, []int{15}
}
func (m *GetTaskSlotsResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_GetTaskSlotsResponse.Unmarshal(m, b)
}
func (m *GetTaskSlotsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
return xxx_messageInfo_GetTaskSlotsResponse.Marshal(b, m, deterministic)
}
func (m *GetTaskSlotsResponse) XXX_Merge(src proto.Message) {
xxx_messageInfo_GetTaskSlotsResponse.Merge(m, src)
}
func (m *GetTaskSlotsResponse) XXX_Size() int {
return xxx_messageInfo_GetTaskSlotsResponse.Size(m)
}
func (m *GetTaskSlotsResponse) XXX_DiscardUnknown() {
xxx_messageInfo_GetTaskSlotsResponse.DiscardUnknown(m)
}
var xxx_messageInfo_GetTaskSlotsResponse proto.InternalMessageInfo
func (m *GetTaskSlotsResponse) GetStatus() *commonpb.Status {
if m != nil {
return m.Status
}
return nil
}
func (m *GetTaskSlotsResponse) GetSlots() int64 {
if m != nil {
return m.Slots
}
return 0
}
func init() { func init() {
proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.index.RegisterNodeRequest") proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.index.RegisterNodeRequest")
proto.RegisterType((*RegisterNodeResponse)(nil), "milvus.proto.index.RegisterNodeResponse") proto.RegisterType((*RegisterNodeResponse)(nil), "milvus.proto.index.RegisterNodeResponse")
@ -885,82 +963,88 @@ func init() {
proto.RegisterType((*IndexMeta)(nil), "milvus.proto.index.IndexMeta") proto.RegisterType((*IndexMeta)(nil), "milvus.proto.index.IndexMeta")
proto.RegisterType((*DropIndexRequest)(nil), "milvus.proto.index.DropIndexRequest") proto.RegisterType((*DropIndexRequest)(nil), "milvus.proto.index.DropIndexRequest")
proto.RegisterType((*RemoveIndexRequest)(nil), "milvus.proto.index.RemoveIndexRequest") proto.RegisterType((*RemoveIndexRequest)(nil), "milvus.proto.index.RemoveIndexRequest")
proto.RegisterType((*GetTaskSlotsRequest)(nil), "milvus.proto.index.GetTaskSlotsRequest")
proto.RegisterType((*GetTaskSlotsResponse)(nil), "milvus.proto.index.GetTaskSlotsResponse")
} }
func init() { proto.RegisterFile("index_coord.proto", fileDescriptor_f9e019eb3fda53c2) } func init() { proto.RegisterFile("index_coord.proto", fileDescriptor_f9e019eb3fda53c2) }
var fileDescriptor_f9e019eb3fda53c2 = []byte{ var fileDescriptor_f9e019eb3fda53c2 = []byte{
// 1118 bytes of a gzipped FileDescriptorProto // 1176 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x57, 0xcd, 0x6e, 0xdb, 0xc6, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x57, 0xcd, 0x6e, 0xdb, 0x46,
0x13, 0x37, 0x4d, 0x5b, 0x1f, 0x23, 0xc5, 0xff, 0x78, 0xff, 0x69, 0xc0, 0x28, 0x09, 0x22, 0xb3, 0x10, 0x0e, 0x4d, 0x5b, 0x3f, 0x23, 0xc5, 0x8d, 0x37, 0x3f, 0x60, 0x94, 0x04, 0x51, 0x98, 0xa4,
0x49, 0xa3, 0x16, 0x89, 0x6c, 0x28, 0x4d, 0x7b, 0x2a, 0xd0, 0xda, 0x42, 0x0c, 0xa1, 0x70, 0x60, 0x51, 0x8b, 0x44, 0x0e, 0x94, 0xa6, 0x3d, 0x15, 0x68, 0x6d, 0x21, 0x86, 0x50, 0x24, 0x30, 0xd6,
0xac, 0x8d, 0x1e, 0x0a, 0x14, 0xc2, 0x5a, 0x1c, 0xd9, 0x8b, 0xf0, 0x43, 0xe6, 0xae, 0x9c, 0xda, 0x46, 0x0e, 0x05, 0x0a, 0x61, 0x2d, 0x8e, 0xec, 0x85, 0xf9, 0x23, 0x73, 0x57, 0x4e, 0xed, 0x73,
0xc7, 0xa2, 0xf7, 0xde, 0x92, 0x47, 0xe8, 0x23, 0xf4, 0xd8, 0x67, 0xe8, 0x1b, 0x15, 0x5c, 0x2e, 0xaf, 0x45, 0x6f, 0xcd, 0x23, 0xf4, 0x11, 0x7a, 0xec, 0x33, 0xf4, 0x79, 0x7a, 0x29, 0xb8, 0x5c,
0x29, 0x52, 0xa2, 0x6c, 0xb9, 0x6e, 0xda, 0x4b, 0x6f, 0x9c, 0xd9, 0xdf, 0xcc, 0xec, 0xfc, 0x76, 0x52, 0xa4, 0x44, 0xd9, 0x72, 0x5d, 0xf7, 0xd4, 0x1b, 0x67, 0xf6, 0x9b, 0x9f, 0xfd, 0x76, 0x66,
0x66, 0x76, 0x09, 0xeb, 0xdc, 0x77, 0xf0, 0xc7, 0xfe, 0x20, 0x08, 0x42, 0xa7, 0x3d, 0x0a, 0x03, 0x76, 0x09, 0x6b, 0xdc, 0x77, 0xf0, 0xc7, 0xfe, 0x20, 0x08, 0x42, 0xa7, 0x3d, 0x0a, 0x03, 0x19,
0x19, 0x10, 0xe2, 0x71, 0xf7, 0x6c, 0x2c, 0x62, 0xa9, 0xad, 0xd6, 0x1b, 0xf5, 0x41, 0xe0, 0x79, 0x10, 0xe2, 0x71, 0xf7, 0x78, 0x2c, 0x62, 0xa9, 0xad, 0xd6, 0x1b, 0xf5, 0x41, 0xe0, 0x79, 0x81,
0x81, 0x1f, 0xeb, 0x1a, 0x6b, 0xdc, 0x97, 0x18, 0xfa, 0xcc, 0xd5, 0x72, 0x3d, 0x6b, 0xd1, 0xa8, 0x1f, 0xeb, 0x1a, 0xab, 0xdc, 0x97, 0x18, 0xfa, 0xcc, 0xd5, 0x72, 0x3d, 0x6b, 0xd1, 0xa8, 0x8b,
0x8b, 0xc1, 0x09, 0x7a, 0x2c, 0x96, 0xec, 0xf7, 0x06, 0xfc, 0x9f, 0xe2, 0x31, 0x17, 0x12, 0xc3, 0xc1, 0x01, 0x7a, 0x2c, 0x96, 0xec, 0x8f, 0x06, 0xdc, 0xa4, 0xb8, 0xcf, 0x85, 0xc4, 0xf0, 0x5d,
0xd7, 0x81, 0x83, 0x14, 0x4f, 0xc7, 0x28, 0x24, 0xd9, 0x82, 0x95, 0x23, 0x26, 0xd0, 0x32, 0x9a, 0xe0, 0x20, 0xc5, 0xa3, 0x31, 0x0a, 0x49, 0x5e, 0xc2, 0xf2, 0x1e, 0x13, 0x68, 0x19, 0x4d, 0xa3,
0x46, 0xab, 0xd6, 0x79, 0xd0, 0xce, 0x05, 0xd5, 0xd1, 0xf6, 0xc4, 0xf1, 0x36, 0x13, 0x48, 0x15, 0x55, 0xeb, 0xdc, 0x6f, 0xe7, 0x82, 0xea, 0x68, 0x6f, 0xc5, 0xfe, 0x06, 0x13, 0x48, 0x15, 0x92,
0x92, 0x7c, 0x01, 0x65, 0xe6, 0x38, 0x21, 0x0a, 0x61, 0x2d, 0x5f, 0x62, 0xf4, 0x4d, 0x8c, 0xa1, 0x7c, 0x09, 0x65, 0xe6, 0x38, 0x21, 0x0a, 0x61, 0x2d, 0x9d, 0x61, 0xf4, 0x6d, 0x8c, 0xa1, 0x09,
0x09, 0x98, 0xdc, 0x85, 0x92, 0x1f, 0x38, 0xd8, 0xeb, 0x5a, 0x66, 0xd3, 0x68, 0x99, 0x54, 0x4b, 0x98, 0xdc, 0x81, 0x92, 0x1f, 0x38, 0xd8, 0xeb, 0x5a, 0x66, 0xd3, 0x68, 0x99, 0x54, 0x4b, 0xf6,
0xf6, 0x2f, 0x06, 0xdc, 0xc9, 0xef, 0x4c, 0x8c, 0x02, 0x5f, 0x20, 0x79, 0x01, 0x25, 0x21, 0x99, 0x2f, 0x06, 0xdc, 0xca, 0x67, 0x26, 0x46, 0x81, 0x2f, 0x90, 0xbc, 0x82, 0x92, 0x90, 0x4c, 0x8e,
0x1c, 0x0b, 0xbd, 0xb9, 0xfb, 0x85, 0x71, 0x0e, 0x14, 0x84, 0x6a, 0x28, 0xd9, 0x86, 0x1a, 0xf7, 0x85, 0x4e, 0xee, 0x5e, 0x61, 0x9c, 0x1d, 0x05, 0xa1, 0x1a, 0x4a, 0x36, 0xa0, 0xc6, 0x7d, 0x2e,
0xb9, 0xec, 0x8f, 0x58, 0xc8, 0xbc, 0x64, 0x87, 0x1b, 0xed, 0x29, 0x2e, 0x35, 0x6d, 0x3d, 0x9f, 0xfb, 0x23, 0x16, 0x32, 0x2f, 0xc9, 0xf0, 0x51, 0x7b, 0x8a, 0x4b, 0x4d, 0x5b, 0xcf, 0xe7, 0x72,
0xcb, 0x7d, 0x05, 0xa4, 0xc0, 0xd3, 0x6f, 0xfb, 0x2b, 0xf8, 0x68, 0x17, 0x65, 0x2f, 0x62, 0x3c, 0x5b, 0x01, 0x29, 0xf0, 0xf4, 0xdb, 0xfe, 0x1a, 0x6e, 0x6f, 0xa1, 0xec, 0x45, 0x8c, 0x47, 0xde,
0xf2, 0x8e, 0x22, 0x21, 0xeb, 0x31, 0xdc, 0x52, 0xe7, 0xb0, 0x3d, 0xe6, 0xae, 0xd3, 0xeb, 0x46, 0x51, 0x24, 0x64, 0x3d, 0x81, 0xeb, 0xea, 0x1c, 0x36, 0xc6, 0xdc, 0x75, 0x7a, 0xdd, 0x28, 0x31,
0x1b, 0x33, 0x5b, 0x26, 0xcd, 0x2b, 0xed, 0xdf, 0x0c, 0xa8, 0x2a, 0xe3, 0x9e, 0x3f, 0x0c, 0xc8, 0xb3, 0x65, 0xd2, 0xbc, 0xd2, 0xfe, 0xdd, 0x80, 0xaa, 0x32, 0xee, 0xf9, 0xc3, 0x80, 0xbc, 0x86,
0x4b, 0x58, 0x8d, 0xb6, 0x16, 0x33, 0xbc, 0xd6, 0x79, 0x54, 0x98, 0xc4, 0x24, 0x16, 0x8d, 0xd1, 0x95, 0x28, 0xb5, 0x98, 0xe1, 0xd5, 0xce, 0xc3, 0xc2, 0x4d, 0x4c, 0x62, 0xd1, 0x18, 0x4d, 0x6c,
0xc4, 0x86, 0x7a, 0xd6, 0xab, 0x4a, 0xc4, 0xa4, 0x39, 0x1d, 0xb1, 0xa0, 0xac, 0xe4, 0x94, 0xd2, 0xa8, 0x67, 0xbd, 0xaa, 0x8d, 0x98, 0x34, 0xa7, 0x23, 0x16, 0x94, 0x95, 0x9c, 0x52, 0x9a, 0x88,
0x44, 0x24, 0x0f, 0x01, 0xe2, 0x82, 0xf2, 0x99, 0x87, 0xd6, 0x4a, 0xd3, 0x68, 0x55, 0x69, 0x55, 0xe4, 0x01, 0x40, 0x5c, 0x50, 0x3e, 0xf3, 0xd0, 0x5a, 0x6e, 0x1a, 0xad, 0x2a, 0xad, 0x2a, 0xcd,
0x69, 0x5e, 0x33, 0x0f, 0xa3, 0xa3, 0x08, 0x91, 0x89, 0xc0, 0xb7, 0x56, 0xd5, 0x92, 0x96, 0xec, 0x3b, 0xe6, 0x61, 0x74, 0x14, 0x21, 0x32, 0x11, 0xf8, 0xd6, 0x8a, 0x5a, 0xd2, 0x92, 0xfd, 0x93,
0x9f, 0x0d, 0xb8, 0x3b, 0x9d, 0xf9, 0x4d, 0x0e, 0xe3, 0x65, 0x6c, 0x84, 0xd1, 0x39, 0x98, 0xad, 0x01, 0x77, 0xa6, 0x77, 0x7e, 0x99, 0xc3, 0x78, 0x1d, 0x1b, 0x61, 0x74, 0x0e, 0x66, 0xab, 0xd6,
0x5a, 0xe7, 0x61, 0x7b, 0xb6, 0xa6, 0xdb, 0x29, 0x55, 0x54, 0x83, 0xed, 0x3f, 0x96, 0x81, 0xec, 0x79, 0xd0, 0x9e, 0xad, 0xe9, 0x76, 0x4a, 0x15, 0xd5, 0x60, 0xfb, 0xcf, 0x25, 0x20, 0x9b, 0x21,
0x84, 0xc8, 0x24, 0xaa, 0xb5, 0x84, 0xfd, 0x69, 0x4a, 0x8c, 0x02, 0x4a, 0xf2, 0x89, 0x2f, 0x4f, 0x32, 0x89, 0x6a, 0x2d, 0x61, 0x7f, 0x9a, 0x12, 0xa3, 0x80, 0x92, 0xfc, 0xc6, 0x97, 0xa6, 0x37,
0x27, 0x3e, 0x9f, 0x31, 0x0b, 0xca, 0x67, 0x18, 0x0a, 0x1e, 0xf8, 0x8a, 0x2e, 0x93, 0x26, 0x22, 0x3e, 0x9f, 0x31, 0x0b, 0xca, 0xc7, 0x18, 0x0a, 0x1e, 0xf8, 0x8a, 0x2e, 0x93, 0x26, 0x22, 0xb9,
0xb9, 0x0f, 0x55, 0x0f, 0x25, 0xeb, 0x8f, 0x98, 0x3c, 0xd1, 0x7c, 0x55, 0x22, 0xc5, 0x3e, 0x93, 0x07, 0x55, 0x0f, 0x25, 0xeb, 0x8f, 0x98, 0x3c, 0xd0, 0x7c, 0x55, 0x22, 0xc5, 0x36, 0x93, 0x07,
0x27, 0x51, 0x3c, 0x87, 0xe9, 0x45, 0x61, 0x95, 0x9a, 0x66, 0x14, 0x2f, 0xd2, 0x44, 0xab, 0xaa, 0x51, 0x3c, 0x87, 0xe9, 0x45, 0x61, 0x95, 0x9a, 0x66, 0x14, 0x2f, 0xd2, 0x44, 0xab, 0xaa, 0x1a,
0x1a, 0xe5, 0xf9, 0x08, 0x93, 0x6a, 0x2c, 0x2b, 0x16, 0x36, 0x0a, 0xa9, 0xfb, 0x16, 0xcf, 0xbf, 0xe5, 0xc9, 0x08, 0x93, 0x6a, 0x2c, 0x2b, 0x16, 0x1e, 0x15, 0x52, 0xf7, 0x1d, 0x9e, 0xbc, 0x67,
0x63, 0xee, 0x18, 0xf7, 0x19, 0x0f, 0x29, 0x44, 0x56, 0x71, 0x35, 0x92, 0xae, 0x4e, 0x3b, 0x71, 0xee, 0x18, 0xb7, 0x19, 0x0f, 0x29, 0x44, 0x56, 0x71, 0x35, 0x92, 0xae, 0xde, 0x76, 0xe2, 0xa4,
0x52, 0x59, 0xd4, 0x49, 0x4d, 0x99, 0xe9, 0x9a, 0x7e, 0x6f, 0xc2, 0x7a, 0x4c, 0xd2, 0x3f, 0x46, 0xb2, 0xa8, 0x93, 0x9a, 0x32, 0xd3, 0x35, 0xfd, 0xd1, 0x84, 0xb5, 0x98, 0xa4, 0xff, 0x8c, 0xd2,
0x69, 0x9e, 0x9b, 0xd5, 0x2b, 0xb8, 0x29, 0xfd, 0x1d, 0xdc, 0x94, 0xff, 0x0a, 0x37, 0xe4, 0x1e, 0x3c, 0x37, 0x2b, 0xe7, 0x70, 0x53, 0xfa, 0x37, 0xb8, 0x29, 0xff, 0x13, 0x6e, 0xc8, 0x5d, 0xa8,
0x54, 0xfc, 0xb1, 0xd7, 0x0f, 0x83, 0xb7, 0x11, 0xbb, 0x2a, 0x07, 0x7f, 0xec, 0xd1, 0xe0, 0xad, 0xf8, 0x63, 0xaf, 0x1f, 0x06, 0x1f, 0x22, 0x76, 0xd5, 0x1e, 0xfc, 0xb1, 0x47, 0x83, 0x0f, 0x82,
0x20, 0x3b, 0x50, 0x1f, 0x72, 0x74, 0x9d, 0x7e, 0x3c, 0x4c, 0xad, 0xaa, 0x2a, 0xfe, 0x66, 0x3e, 0x6c, 0x42, 0x7d, 0xc8, 0xd1, 0x75, 0xfa, 0xf1, 0x30, 0xb5, 0xaa, 0xaa, 0xf8, 0x9b, 0xf9, 0x00,
0x80, 0x1e, 0xb4, 0xaf, 0x22, 0xe0, 0x81, 0xfa, 0xa6, 0xb5, 0xe1, 0x44, 0x20, 0x0f, 0xa0, 0x2a, 0x7a, 0xd0, 0xbe, 0x89, 0x80, 0x3b, 0xea, 0x9b, 0xd6, 0x86, 0x13, 0x81, 0xdc, 0x87, 0xaa, 0xc0,
0xf0, 0xd8, 0x43, 0x5f, 0xf6, 0xba, 0x16, 0xa8, 0x00, 0x13, 0x85, 0xed, 0x01, 0xc9, 0x1e, 0xcc, 0x7d, 0x0f, 0x7d, 0xd9, 0xeb, 0x5a, 0xa0, 0x02, 0x4c, 0x14, 0xb6, 0x07, 0x24, 0x7b, 0x30, 0x97,
0x4d, 0xfa, 0x6d, 0x81, 0xa1, 0x61, 0x7f, 0x0d, 0x56, 0xd2, 0xe2, 0xaf, 0xb8, 0x8b, 0xea, 0x2c, 0xe9, 0xb7, 0x05, 0x86, 0x86, 0xfd, 0x0d, 0x58, 0x49, 0x8b, 0xbf, 0xe1, 0x2e, 0xaa, 0xb3, 0xb8,
0xae, 0x37, 0xdf, 0x7e, 0x37, 0x60, 0x3d, 0x67, 0xaf, 0xe6, 0xdc, 0x87, 0xda, 0x30, 0x69, 0xc1, 0xd8, 0x7c, 0xfb, 0xc3, 0x80, 0xb5, 0x9c, 0xbd, 0x9a, 0x73, 0x57, 0x95, 0x30, 0x69, 0xc1, 0x8d,
0xed, 0xf8, 0x8c, 0x87, 0xdc, 0x45, 0x5d, 0x4c, 0xa6, 0x2a, 0xa6, 0x35, 0x9e, 0xcb, 0x82, 0x3c, 0xf8, 0x8c, 0x87, 0xdc, 0x45, 0x5d, 0x4c, 0xa6, 0x2a, 0xa6, 0x55, 0x9e, 0xdb, 0x05, 0x79, 0x06,
0x85, 0xff, 0x09, 0x0c, 0x39, 0x73, 0xf9, 0x05, 0x3a, 0x7d, 0xc1, 0x2f, 0xe2, 0xd1, 0xb7, 0x42, 0x9f, 0x08, 0x0c, 0x39, 0x73, 0xf9, 0x29, 0x3a, 0x7d, 0xc1, 0x4f, 0xe3, 0xd1, 0xb7, 0x4c, 0x57,
0xd7, 0x26, 0xea, 0x03, 0x7e, 0x81, 0xf6, 0x3b, 0x03, 0xee, 0x15, 0x90, 0x70, 0x13, 0xea, 0xbb, 0x27, 0xea, 0x1d, 0x7e, 0x8a, 0xf6, 0xaf, 0x06, 0xdc, 0x2d, 0x20, 0xe1, 0x32, 0xd4, 0x77, 0x01,
0x00, 0x99, 0xfd, 0xc5, 0xe3, 0xee, 0xc9, 0xdc, 0x71, 0x97, 0x65, 0x8e, 0x56, 0x87, 0xc9, 0x16, 0x32, 0xf9, 0xc5, 0xe3, 0xee, 0xe9, 0xdc, 0x71, 0x97, 0x65, 0x8e, 0x56, 0x87, 0x49, 0x0a, 0xf6,
0xec, 0x9f, 0x4c, 0x7d, 0x75, 0xec, 0xa1, 0x64, 0x0b, 0x75, 0x67, 0x7a, 0xbd, 0x2c, 0x5f, 0xeb, 0xcf, 0xa6, 0xbe, 0x3a, 0xde, 0xa2, 0x64, 0x0b, 0x75, 0x67, 0x7a, 0xbd, 0x2c, 0x5d, 0xe8, 0x7a,
0x7a, 0x79, 0x04, 0xb5, 0x21, 0xe3, 0x6e, 0x5f, 0x5f, 0x03, 0xa6, 0xea, 0x6a, 0x88, 0x54, 0x54, 0x79, 0x08, 0xb5, 0x21, 0xe3, 0x6e, 0x5f, 0x5f, 0x03, 0xa6, 0xea, 0x6a, 0x88, 0x54, 0x54, 0x69,
0x69, 0xc8, 0x97, 0x60, 0x86, 0x78, 0xaa, 0xf8, 0x9b, 0x93, 0xc8, 0xcc, 0x34, 0xa1, 0x91, 0x45, 0xc8, 0x57, 0x60, 0x86, 0x78, 0xa4, 0xf8, 0x9b, 0xb3, 0x91, 0x99, 0x69, 0x42, 0x23, 0x8b, 0xc2,
0xe1, 0x71, 0xad, 0x16, 0x1e, 0xd7, 0x06, 0xd4, 0x3d, 0x16, 0xbe, 0xe9, 0x3b, 0xe8, 0xa2, 0x44, 0xe3, 0x5a, 0x29, 0x3c, 0xae, 0x47, 0x50, 0xf7, 0x58, 0x78, 0xd8, 0x77, 0xd0, 0x45, 0x89, 0x8e,
0xc7, 0x2a, 0x35, 0x8d, 0x56, 0x85, 0xd6, 0x22, 0x5d, 0x37, 0x56, 0x65, 0xde, 0x0c, 0xe5, 0xec, 0x55, 0x6a, 0x1a, 0xad, 0x0a, 0xad, 0x45, 0xba, 0x6e, 0xac, 0xca, 0xbc, 0x19, 0xca, 0xd9, 0x37,
0x9b, 0x21, 0x3b, 0xad, 0x2b, 0xf9, 0x69, 0xdd, 0x80, 0x4a, 0x88, 0x83, 0xf3, 0x81, 0x8b, 0x8e, 0x03, 0x79, 0xac, 0x0b, 0xb5, 0x9f, 0xcc, 0xec, 0x4a, 0x86, 0x9a, 0xf7, 0x7a, 0x70, 0x37, 0xa0,
0x6a, 0xd6, 0x0a, 0x4d, 0x65, 0xf2, 0x04, 0x26, 0x85, 0x10, 0x97, 0x07, 0xa8, 0xf2, 0xb8, 0x95, 0x12, 0xe2, 0xe0, 0x64, 0xe0, 0xa2, 0xa3, 0xfa, 0xb6, 0x42, 0x53, 0x99, 0x3c, 0x85, 0x49, 0x4d,
0x6a, 0x55, 0x75, 0x3c, 0x83, 0xdb, 0xdd, 0x30, 0x18, 0xe5, 0x06, 0x65, 0x66, 0xca, 0x19, 0xb9, 0xc4, 0x95, 0x02, 0xaa, 0x52, 0xae, 0xa7, 0x5a, 0x55, 0x28, 0xcf, 0xe1, 0x46, 0x37, 0x0c, 0x46,
0x29, 0x67, 0x6f, 0x01, 0xa1, 0xe8, 0x05, 0x67, 0xf9, 0xbb, 0xaa, 0x01, 0x95, 0xa3, 0x7c, 0x13, 0xb9, 0x99, 0x99, 0x19, 0x78, 0x46, 0x6e, 0xe0, 0xd9, 0x2f, 0x81, 0x50, 0xf4, 0x82, 0xe3, 0xfc,
0xa5, 0x72, 0xe7, 0xd7, 0x32, 0x80, 0x02, 0xef, 0x44, 0xaf, 0x3d, 0x32, 0x02, 0xb2, 0x8b, 0x72, 0xb5, 0xd5, 0x80, 0xca, 0x5e, 0xbe, 0x9f, 0x52, 0xd9, 0xbe, 0x0d, 0x37, 0xb7, 0x50, 0xee, 0x32,
0x27, 0xf0, 0x46, 0x81, 0x8f, 0xbe, 0x8c, 0xef, 0x5d, 0xb2, 0x35, 0xe7, 0xc9, 0x32, 0x0b, 0xd5, 0x71, 0xb8, 0xe3, 0x06, 0x32, 0xe9, 0x43, 0x9b, 0xc1, 0xad, 0xbc, 0xfa, 0x32, 0x95, 0x79, 0x0b,
0x21, 0x1b, 0x9f, 0xcc, 0xb1, 0x98, 0x82, 0xdb, 0x4b, 0xc4, 0x53, 0x11, 0x0f, 0xb9, 0x87, 0x87, 0x56, 0x44, 0xe4, 0x45, 0x37, 0x57, 0x2c, 0x74, 0x7e, 0x2b, 0x03, 0xa8, 0x34, 0x37, 0xa3, 0x27,
0x7c, 0xf0, 0x66, 0xe7, 0x84, 0xf9, 0x3e, 0xba, 0x97, 0x45, 0x9c, 0x82, 0x26, 0x11, 0x3f, 0xce, 0x27, 0x19, 0x01, 0xd9, 0x42, 0xb9, 0x19, 0x78, 0xa3, 0xc0, 0x47, 0x5f, 0xc6, 0x97, 0x3f, 0x79,
0x5b, 0x68, 0xe1, 0x40, 0x86, 0xdc, 0x3f, 0x4e, 0xba, 0xc9, 0x5e, 0x22, 0xa7, 0x70, 0x67, 0x17, 0x39, 0xe7, 0xdd, 0x34, 0x0b, 0xd5, 0x99, 0x37, 0x3e, 0x9d, 0x63, 0x31, 0x05, 0xb7, 0xaf, 0x11,
0x55, 0x74, 0x2e, 0x24, 0x1f, 0x88, 0x24, 0x60, 0x67, 0x7e, 0xc0, 0x19, 0xf0, 0x35, 0x43, 0xfe, 0x4f, 0x45, 0xdc, 0xe5, 0x1e, 0xee, 0xf2, 0xc1, 0xe1, 0xe6, 0x01, 0xf3, 0x7d, 0x74, 0xcf, 0x8a,
0x00, 0x30, 0x29, 0x4f, 0xb2, 0x58, 0xf9, 0xce, 0x12, 0x38, 0x0d, 0x4b, 0xdd, 0x73, 0x58, 0xcb, 0x38, 0x05, 0x4d, 0x22, 0x3e, 0xce, 0x5b, 0x68, 0x61, 0x47, 0x86, 0xdc, 0xdf, 0x4f, 0x88, 0xb3,
0x3f, 0x93, 0xc8, 0xa7, 0x45, 0xb6, 0x85, 0x8f, 0xc8, 0xc6, 0x67, 0x8b, 0x40, 0xd3, 0x50, 0x21, 0xaf, 0x91, 0x23, 0x45, 0x69, 0x14, 0x9d, 0x0b, 0xc9, 0x07, 0x22, 0x09, 0xd8, 0x99, 0x1f, 0x70,
0xac, 0xcf, 0x4c, 0x2a, 0xf2, 0xec, 0x32, 0x17, 0xd3, 0x53, 0xbd, 0xf1, 0x7c, 0x41, 0x74, 0x1a, 0x06, 0x7c, 0xc1, 0x90, 0x3f, 0x00, 0x4c, 0x7a, 0x84, 0x2c, 0xd6, 0x43, 0xb3, 0x04, 0x4e, 0xc3,
0x73, 0x1f, 0xaa, 0x69, 0x03, 0x90, 0xc7, 0x45, 0xd6, 0xd3, 0xfd, 0xd1, 0xb8, 0x6c, 0x46, 0xda, 0x52, 0xf7, 0x1c, 0x56, 0xf3, 0x6f, 0x35, 0xf2, 0x59, 0x91, 0x6d, 0xe1, 0x4b, 0xb6, 0xf1, 0xf9,
0x4b, 0xe4, 0x10, 0x6a, 0x99, 0x26, 0x21, 0x85, 0x4c, 0xcf, 0x76, 0xd1, 0x55, 0x5e, 0xfb, 0x00, 0x22, 0xd0, 0x34, 0x54, 0x08, 0x6b, 0x33, 0xe3, 0x92, 0x3c, 0x3f, 0xcb, 0xc5, 0xf4, 0xd5, 0xd2,
0xbb, 0x28, 0xf7, 0x50, 0x86, 0x7c, 0x20, 0xa6, 0x9d, 0x6a, 0x61, 0x02, 0x48, 0x9c, 0x3e, 0xbd, 0x78, 0xb1, 0x20, 0x3a, 0x8d, 0xb9, 0x0d, 0xd5, 0xb4, 0xf5, 0xc8, 0x93, 0x22, 0xeb, 0xe9, 0xce,
0x12, 0x97, 0x10, 0xd1, 0x79, 0xb7, 0xa2, 0xc7, 0x71, 0xf4, 0x5f, 0xf2, 0x5f, 0xa3, 0x7e, 0x80, 0x6c, 0x9c, 0xd5, 0x0e, 0xf6, 0x35, 0xb2, 0x0b, 0xb5, 0x4c, 0x7b, 0x92, 0x42, 0xa6, 0x67, 0xfb,
0x46, 0x3d, 0x84, 0x5a, 0xe6, 0xa5, 0x5f, 0x5c, 0x18, 0xb3, 0xbf, 0x02, 0xff, 0x76, 0x61, 0x6c, 0xf7, 0x3c, 0xaf, 0x7d, 0x80, 0x2d, 0x94, 0x6f, 0x51, 0x86, 0x7c, 0x20, 0xa6, 0x9d, 0x6a, 0x61,
0x7f, 0xfe, 0x7d, 0xe7, 0x98, 0xcb, 0x93, 0xf1, 0x51, 0x14, 0x7a, 0x33, 0x46, 0x3e, 0xe7, 0x81, 0x02, 0x48, 0x9c, 0x3e, 0x3b, 0x17, 0x97, 0x10, 0xd1, 0xf9, 0x6b, 0x59, 0xdf, 0x09, 0xd1, 0xcf,
0xfe, 0xda, 0x4c, 0x18, 0xda, 0x54, 0x9e, 0x36, 0x55, 0x1a, 0xa3, 0xa3, 0xa3, 0x92, 0x12, 0x5f, 0xd1, 0xff, 0x8d, 0x7a, 0x05, 0x8d, 0xba, 0x0b, 0xb5, 0xcc, 0xef, 0x46, 0x71, 0x61, 0xcc, 0xfe,
0xfc, 0x19, 0x00, 0x00, 0xff, 0xff, 0xa7, 0x8e, 0x84, 0xfb, 0xed, 0x0f, 0x00, 0x00, 0x8f, 0x9c, 0x57, 0x18, 0x03, 0xa8, 0x67, 0x87, 0x38, 0x79, 0x36, 0xa7, 0x03, 0xa6, 0xa7, 0x7f,
0xa3, 0x75, 0x3e, 0x30, 0x4d, 0xfd, 0xaa, 0xab, 0x6f, 0xe3, 0x8b, 0xef, 0x3b, 0xfb, 0x5c, 0x1e,
0x8c, 0xf7, 0xa2, 0xfd, 0xad, 0xc7, 0xc8, 0x17, 0x3c, 0xd0, 0x5f, 0xeb, 0xc9, 0x31, 0xac, 0x2b,
0x4f, 0xeb, 0x2a, 0xd7, 0xd1, 0xde, 0x5e, 0x49, 0x89, 0xaf, 0xfe, 0x0e, 0x00, 0x00, 0xff, 0xff,
0x71, 0x98, 0x62, 0xf8, 0xd7, 0x10, 0x00, 0x00,
} }
// Reference imports to suppress errors if they are not otherwise used. // Reference imports to suppress errors if they are not otherwise used.
@ -1341,6 +1425,7 @@ type IndexNodeClient interface {
GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
GetTaskSlots(ctx context.Context, in *GetTaskSlotsRequest, opts ...grpc.CallOption) (*GetTaskSlotsResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error)
} }
@ -1389,6 +1474,15 @@ func (c *indexNodeClient) CreateIndex(ctx context.Context, in *CreateIndexReques
return out, nil return out, nil
} }
func (c *indexNodeClient) GetTaskSlots(ctx context.Context, in *GetTaskSlotsRequest, opts ...grpc.CallOption) (*GetTaskSlotsResponse, error) {
out := new(GetTaskSlotsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.index.IndexNode/GetTaskSlots", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *indexNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { func (c *indexNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
out := new(milvuspb.GetMetricsResponse) out := new(milvuspb.GetMetricsResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.index.IndexNode/GetMetrics", in, out, opts...) err := c.cc.Invoke(ctx, "/milvus.proto.index.IndexNode/GetMetrics", in, out, opts...)
@ -1404,6 +1498,7 @@ type IndexNodeServer interface {
GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) GetTimeTickChannel(context.Context, *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error)
GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) GetStatisticsChannel(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error)
CreateIndex(context.Context, *CreateIndexRequest) (*commonpb.Status, error) CreateIndex(context.Context, *CreateIndexRequest) (*commonpb.Status, error)
GetTaskSlots(context.Context, *GetTaskSlotsRequest) (*GetTaskSlotsResponse, error)
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy // https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
} }
@ -1424,6 +1519,9 @@ func (*UnimplementedIndexNodeServer) GetStatisticsChannel(ctx context.Context, r
func (*UnimplementedIndexNodeServer) CreateIndex(ctx context.Context, req *CreateIndexRequest) (*commonpb.Status, error) { func (*UnimplementedIndexNodeServer) CreateIndex(ctx context.Context, req *CreateIndexRequest) (*commonpb.Status, error) {
return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented") return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented")
} }
func (*UnimplementedIndexNodeServer) GetTaskSlots(ctx context.Context, req *GetTaskSlotsRequest) (*GetTaskSlotsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetTaskSlots not implemented")
}
func (*UnimplementedIndexNodeServer) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { func (*UnimplementedIndexNodeServer) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented") return nil, status.Errorf(codes.Unimplemented, "method GetMetrics not implemented")
} }
@ -1504,6 +1602,24 @@ func _IndexNode_CreateIndex_Handler(srv interface{}, ctx context.Context, dec fu
return interceptor(ctx, in, info, handler) return interceptor(ctx, in, info, handler)
} }
func _IndexNode_GetTaskSlots_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(GetTaskSlotsRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(IndexNodeServer).GetTaskSlots(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.index.IndexNode/GetTaskSlots",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(IndexNodeServer).GetTaskSlots(ctx, req.(*GetTaskSlotsRequest))
}
return interceptor(ctx, in, info, handler)
}
func _IndexNode_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { func _IndexNode_GetMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.GetMetricsRequest) in := new(milvuspb.GetMetricsRequest)
if err := dec(in); err != nil { if err := dec(in); err != nil {
@ -1542,6 +1658,10 @@ var _IndexNode_serviceDesc = grpc.ServiceDesc{
MethodName: "CreateIndex", MethodName: "CreateIndex",
Handler: _IndexNode_CreateIndex_Handler, Handler: _IndexNode_CreateIndex_Handler,
}, },
{
MethodName: "GetTaskSlots",
Handler: _IndexNode_GetTaskSlots_Handler,
},
{ {
MethodName: "GetMetrics", MethodName: "GetMetrics",
Handler: _IndexNode_GetMetrics_Handler, Handler: _IndexNode_GetMetrics_Handler,

View File

@ -159,24 +159,28 @@ func (lcm *LocalChunkManager) MultiRead(filePaths []string) ([][]byte, error) {
return results, el return results, el
} }
func (lcm *LocalChunkManager) ListWithPrefix(prefix string) ([]string, error) { func (lcm *LocalChunkManager) ListWithPrefix(prefix string, recursive bool) ([]string, error) {
var filePaths []string if recursive {
absPrefix := path.Join(lcm.localPath, prefix) var filePaths []string
dir := filepath.Dir(absPrefix) absPrefix := path.Join(lcm.localPath, prefix)
err := filepath.Walk(dir, func(filePath string, f os.FileInfo, err error) error { dir := filepath.Dir(absPrefix)
if strings.HasPrefix(filePath, absPrefix) && !f.IsDir() { err := filepath.Walk(dir, func(filePath string, f os.FileInfo, err error) error {
filePaths = append(filePaths, strings.TrimPrefix(filePath, lcm.localPath)) if strings.HasPrefix(filePath, absPrefix) && !f.IsDir() {
filePaths = append(filePaths, strings.TrimPrefix(filePath, lcm.localPath))
}
return nil
})
if err != nil {
return nil, err
} }
return nil return filePaths, nil
})
if err != nil {
return nil, err
} }
return filePaths, nil absPrefix := path.Join(lcm.localPath, prefix+"*")
return filepath.Glob(absPrefix)
} }
func (lcm *LocalChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte, error) { func (lcm *LocalChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte, error) {
filePaths, err := lcm.ListWithPrefix(prefix) filePaths, err := lcm.ListWithPrefix(prefix, true)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -248,7 +252,7 @@ func (lcm *LocalChunkManager) MultiRemove(filePaths []string) error {
} }
func (lcm *LocalChunkManager) RemoveWithPrefix(prefix string) error { func (lcm *LocalChunkManager) RemoveWithPrefix(prefix string) error {
filePaths, err := lcm.ListWithPrefix(prefix) filePaths, err := lcm.ListWithPrefix(prefix, true)
if err != nil { if err != nil {
return err return err
} }

View File

@ -17,6 +17,7 @@
package storage package storage
import ( import (
"fmt"
"path" "path"
"testing" "testing"
@ -391,13 +392,60 @@ func TestLocalCM(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
pathPrefix := path.Join(testPrefix, "a") pathPrefix := path.Join(testPrefix, "a")
r, err := testCM.ListWithPrefix(pathPrefix) r, err := testCM.ListWithPrefix(pathPrefix, true)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, len(r), 2) assert.Equal(t, len(r), 2)
testCM.RemoveWithPrefix(testPrefix) testCM.RemoveWithPrefix(testPrefix)
r, err = testCM.ListWithPrefix(pathPrefix) r, err = testCM.ListWithPrefix(pathPrefix, true)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, len(r), 0) assert.Equal(t, len(r), 0)
}) })
t.Run("test ListWithPrefix", func(t *testing.T) {
testPrefix := "prefix-ListWithPrefix"
testCM := NewLocalChunkManager(RootPath(localPath))
defer testCM.RemoveWithPrefix(testPrefix)
key := path.Join(testPrefix, "abc", "def")
value := []byte("a")
err := testCM.Write(key, value)
assert.NoError(t, err)
key = path.Join(testPrefix, "abc", "deg")
err = testCM.Write(key, value)
assert.NoError(t, err)
key = path.Join(testPrefix, "abd")
err = testCM.Write(key, value)
assert.NoError(t, err)
key = path.Join(testPrefix, "bcd")
err = testCM.Write(key, value)
assert.NoError(t, err)
dirs, err := testCM.ListWithPrefix(testPrefix+"/", false)
assert.Nil(t, err)
fmt.Println(dirs)
assert.Equal(t, 3, len(dirs))
testPrefix2 := path.Join(testPrefix, "a")
dirs, err = testCM.ListWithPrefix(testPrefix2, false)
assert.Nil(t, err)
assert.Equal(t, 2, len(dirs))
dirs, err = testCM.ListWithPrefix(testPrefix2, false)
assert.Nil(t, err)
assert.Equal(t, 2, len(dirs))
err = testCM.RemoveWithPrefix(testPrefix)
assert.NoError(t, err)
dirs, err = testCM.ListWithPrefix(testPrefix, false)
assert.NoError(t, err)
fmt.Println(dirs)
// dir still exist
assert.Equal(t, 1, len(dirs))
})
} }

View File

@ -191,6 +191,10 @@ func (mcm *MinioChunkManager) Read(filePath string) ([]byte, error) {
data, err := ioutil.ReadAll(object) data, err := ioutil.ReadAll(object)
if err != nil { if err != nil {
errResponse := minio.ToErrorResponse(err)
if errResponse.Code == "NoSuchKey" {
return nil, errors.New("NoSuchKey")
}
log.Warn("failed to read object", zap.String("path", filePath), zap.Error(err)) log.Warn("failed to read object", zap.String("path", filePath), zap.Error(err))
return nil, err return nil, err
} }
@ -215,7 +219,7 @@ func (mcm *MinioChunkManager) MultiRead(keys []string) ([][]byte, error) {
} }
func (mcm *MinioChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte, error) { func (mcm *MinioChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte, error) {
objectsKeys, err := mcm.ListWithPrefix(prefix) objectsKeys, err := mcm.ListWithPrefix(prefix, true)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -295,8 +299,8 @@ func (mcm *MinioChunkManager) RemoveWithPrefix(prefix string) error {
return nil return nil
} }
func (mcm *MinioChunkManager) ListWithPrefix(prefix string) ([]string, error) { func (mcm *MinioChunkManager) ListWithPrefix(prefix string, recursive bool) ([]string, error) {
objects := mcm.Client.ListObjects(mcm.ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: prefix, Recursive: true}) objects := mcm.Client.ListObjects(mcm.ctx, mcm.bucketName, minio.ListObjectsOptions{Prefix: prefix, Recursive: recursive})
var objectsKeys []string var objectsKeys []string
for object := range objects { for object := range objects {

View File

@ -18,7 +18,6 @@ package storage
import ( import (
"context" "context"
"fmt"
"path" "path"
"strconv" "strconv"
"testing" "testing"
@ -454,20 +453,38 @@ func TestMinIOCM(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
pathPrefix := path.Join(testPrefix, "a") pathPrefix := path.Join(testPrefix, "a")
r, err := testCM.ListWithPrefix(pathPrefix) r, err := testCM.ListWithPrefix(pathPrefix, true)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, len(r), 2) assert.Equal(t, len(r), 2)
key = path.Join(testPrefix, "b", "b", "b")
err = testCM.Write(key, value)
assert.NoError(t, err)
key = path.Join(testPrefix, "b", "a", "b")
err = testCM.Write(key, value)
assert.NoError(t, err)
key = path.Join(testPrefix, "bc", "a", "b")
err = testCM.Write(key, value)
assert.NoError(t, err)
dirs, err := testCM.ListWithPrefix(testPrefix+"/", false)
assert.NoError(t, err)
assert.Equal(t, 3, len(dirs))
dirs, err = testCM.ListWithPrefix(path.Join(testPrefix, "b"), false)
assert.NoError(t, err)
assert.Equal(t, 2, len(dirs))
testCM.RemoveWithPrefix(testPrefix) testCM.RemoveWithPrefix(testPrefix)
r, err = testCM.ListWithPrefix(pathPrefix) r, err = testCM.ListWithPrefix(pathPrefix, false)
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, len(r), 0) assert.Equal(t, len(r), 0)
// test wrong prefix // test wrong prefix
b := make([]byte, 2048) b := make([]byte, 2048)
pathWrong := path.Join(testPrefix, string(b)) pathWrong := path.Join(testPrefix, string(b))
_, err = testCM.ListWithPrefix(pathWrong) _, err = testCM.ListWithPrefix(pathWrong, true)
assert.Error(t, err) assert.Error(t, err)
fmt.Println(err)
}) })
} }

View File

@ -41,7 +41,7 @@ type ChunkManager interface {
Reader(filePath string) (FileReader, error) Reader(filePath string) (FileReader, error)
// MultiRead reads @filePath and returns content. // MultiRead reads @filePath and returns content.
MultiRead(filePaths []string) ([][]byte, error) MultiRead(filePaths []string) ([][]byte, error)
ListWithPrefix(prefix string) ([]string, error) ListWithPrefix(prefix string, recursive bool) ([]string, error)
// ReadWithPrefix reads files with same @prefix and returns contents. // ReadWithPrefix reads files with same @prefix and returns contents.
ReadWithPrefix(prefix string) ([]string, [][]byte, error) ReadWithPrefix(prefix string) ([]string, [][]byte, error)
Mmap(filePath string) (*mmap.ReaderAt, error) Mmap(filePath string) (*mmap.ReaderAt, error)

View File

@ -216,7 +216,7 @@ func (vcm *VectorChunkManager) MultiRead(filePaths []string) ([][]byte, error) {
} }
func (vcm *VectorChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte, error) { func (vcm *VectorChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte, error) {
filePaths, err := vcm.ListWithPrefix(prefix) filePaths, err := vcm.ListWithPrefix(prefix, true)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
@ -227,8 +227,8 @@ func (vcm *VectorChunkManager) ReadWithPrefix(prefix string) ([]string, [][]byte
return filePaths, results, nil return filePaths, results, nil
} }
func (vcm *VectorChunkManager) ListWithPrefix(prefix string) ([]string, error) { func (vcm *VectorChunkManager) ListWithPrefix(prefix string, recursive bool) ([]string, error) {
return vcm.vectorStorage.ListWithPrefix(prefix) return vcm.vectorStorage.ListWithPrefix(prefix, recursive)
} }
func (vcm *VectorChunkManager) Mmap(filePath string) (*mmap.ReaderAt, error) { func (vcm *VectorChunkManager) Mmap(filePath string) (*mmap.ReaderAt, error) {
@ -312,7 +312,7 @@ func (vcm *VectorChunkManager) RemoveWithPrefix(prefix string) error {
return err return err
} }
if vcm.cacheEnable { if vcm.cacheEnable {
filePaths, err := vcm.ListWithPrefix(prefix) filePaths, err := vcm.ListWithPrefix(prefix, true)
if err != nil { if err != nil {
return err return err
} }

View File

@ -310,6 +310,7 @@ type IndexNode interface {
// CreateIndex receives request from IndexCoordinator to build an index. // CreateIndex receives request from IndexCoordinator to build an index.
// Index building is asynchronous, so when an index building request comes, IndexNode records the task and returns. // Index building is asynchronous, so when an index building request comes, IndexNode records the task and returns.
CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error)
GetTaskSlots(ctx context.Context, req *indexpb.GetTaskSlotsRequest) (*indexpb.GetTaskSlotsResponse, error)
// GetMetrics gets the metrics about IndexNode. // GetMetrics gets the metrics about IndexNode.
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)

View File

@ -58,7 +58,7 @@ func (mc *MockChunkManager) MultiRead(filePaths []string) ([][]byte, error) {
return nil, nil return nil, nil
} }
func (mc *MockChunkManager) ListWithPrefix(prefix string) ([]string, error) { func (mc *MockChunkManager) ListWithPrefix(prefix string, recursive bool) ([]string, error) {
return nil, nil return nil, nil
} }

View File

@ -52,3 +52,7 @@ func (m *GrpcIndexNodeClient) CreateIndex(ctx context.Context, in *indexpb.Creat
func (m *GrpcIndexNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { func (m *GrpcIndexNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
return &milvuspb.GetMetricsResponse{}, m.Err return &milvuspb.GetMetricsResponse{}, m.Err
} }
func (m *GrpcIndexNodeClient) GetTaskSlots(ctx context.Context, in *indexpb.GetTaskSlotsRequest, opts ...grpc.CallOption) (*indexpb.GetTaskSlotsResponse, error) {
return &indexpb.GetTaskSlotsResponse{}, m.Err
}

View File

@ -1194,6 +1194,8 @@ type indexCoordConfig struct {
IndexStorageRootPath string IndexStorageRootPath string
GCInterval time.Duration
CreatedTime time.Time CreatedTime time.Time
UpdatedTime time.Time UpdatedTime time.Time
} }
@ -1202,6 +1204,7 @@ func (p *indexCoordConfig) init(base *BaseTable) {
p.Base = base p.Base = base
p.initIndexStorageRootPath() p.initIndexStorageRootPath()
p.initGCInterval()
} }
// initIndexStorageRootPath initializes the root path of index files. // initIndexStorageRootPath initializes the root path of index files.
@ -1213,6 +1216,10 @@ func (p *indexCoordConfig) initIndexStorageRootPath() {
p.IndexStorageRootPath = path.Join(rootPath, "index_files") p.IndexStorageRootPath = path.Join(rootPath, "index_files")
} }
func (p *indexCoordConfig) initGCInterval() {
p.GCInterval = time.Duration(p.Base.ParseInt64WithDefault("indexCoord.gc.interval", 60*10)) * time.Second
}
/////////////////////////////////////////////////////////////////////////////// ///////////////////////////////////////////////////////////////////////////////
// --- indexnode --- // --- indexnode ---
type indexNodeConfig struct { type indexNodeConfig struct {
@ -1227,6 +1234,7 @@ type indexNodeConfig struct {
Alias string Alias string
IndexStorageRootPath string IndexStorageRootPath string
BuildParallel int
CreatedTime time.Time CreatedTime time.Time
UpdatedTime time.Time UpdatedTime time.Time
@ -1236,6 +1244,7 @@ func (p *indexNodeConfig) init(base *BaseTable) {
p.Base = base p.Base = base
p.NodeID.Store(UniqueID(0)) p.NodeID.Store(UniqueID(0))
p.initIndexStorageRootPath() p.initIndexStorageRootPath()
p.initBuildParallel()
} }
// InitAlias initializes an alias for the IndexNode role. // InitAlias initializes an alias for the IndexNode role.
@ -1251,6 +1260,10 @@ func (p *indexNodeConfig) initIndexStorageRootPath() {
p.IndexStorageRootPath = path.Join(rootPath, "index_files") p.IndexStorageRootPath = path.Join(rootPath, "index_files")
} }
func (p *indexNodeConfig) initBuildParallel() {
p.BuildParallel = p.Base.ParseIntWithDefault("indexNode.scheduler.buildParallel", 1)
}
func (p *indexNodeConfig) SetNodeID(id UniqueID) { func (p *indexNodeConfig) SetNodeID(id UniqueID) {
p.NodeID.Store(id) p.NodeID.Store(id)
} }