mirror of https://github.com/milvus-io/milvus.git
Only store segmentID for flushed segments event (#19075)
Signed-off-by: cai.zhang <cai.zhang@zilliz.com> Signed-off-by: cai.zhang <cai.zhang@zilliz.com>pull/19148/head
parent
eabdc2b114
commit
e65ba6331e
|
@ -39,3 +39,7 @@ func msgDataCoordIsUnhealthy(coordID UniqueID) string {
|
|||
func errDataCoordIsUnhealthy(coordID UniqueID) error {
|
||||
return errors.New(msgDataCoordIsUnhealthy(coordID))
|
||||
}
|
||||
|
||||
func msgSegmentNotFound(segID UniqueID) string {
|
||||
return fmt.Sprintf("failed to get segment %d", segID)
|
||||
}
|
||||
|
|
|
@ -37,3 +37,8 @@ func TestErrDataCoordIsUnhealthy(t *testing.T) {
|
|||
log.Info("TestErrDataCoordIsUnhealthy", zap.Error(errDataCoordIsUnhealthy(nodeID)))
|
||||
}
|
||||
}
|
||||
|
||||
func TestErrSegmentNotFound(t *testing.T) {
|
||||
segID := UniqueID(435846569243121068)
|
||||
log.Info("TestErrSegmentNotFound", zap.String("msg", msgSegmentNotFound(segID)))
|
||||
}
|
||||
|
|
|
@ -350,14 +350,14 @@ func (s *Server) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoR
|
|||
info = s.meta.GetAllSegment(id)
|
||||
if info == nil {
|
||||
log.Warn("failed to get segment, this may have been cleaned", zap.Int64("segmentID", id))
|
||||
resp.Status.Reason = fmt.Sprintf("failed to get segment %d", id)
|
||||
resp.Status.Reason = msgSegmentNotFound(id)
|
||||
return resp, nil
|
||||
}
|
||||
infos = append(infos, info.SegmentInfo)
|
||||
} else {
|
||||
info = s.meta.GetSegment(id)
|
||||
if info == nil {
|
||||
resp.Status.Reason = fmt.Sprintf("failed to get segment %d", id)
|
||||
resp.Status.Reason = msgSegmentNotFound(id)
|
||||
return resp, nil
|
||||
}
|
||||
infos = append(infos, info.SegmentInfo)
|
||||
|
|
|
@ -22,8 +22,9 @@ import (
|
|||
)
|
||||
|
||||
var (
|
||||
ErrCompareVersion = errors.New("failed to save meta in etcd because version compare failure")
|
||||
ErrNotIndexExist = errors.New("there is no index")
|
||||
ErrCompareVersion = errors.New("failed to save meta in etcd because version compare failure")
|
||||
ErrNotIndexExist = errors.New("there is no index")
|
||||
ErrSegmentNotFound = errors.New("failed to get segment")
|
||||
)
|
||||
|
||||
// errIndexNodeIsNotOnService return an error that the specified IndexNode is not exists.
|
||||
|
@ -39,3 +40,11 @@ func msgIndexCoordIsUnhealthy(coordID UniqueID) string {
|
|||
func errIndexCoordIsUnhealthy(coordID UniqueID) error {
|
||||
return errors.New(msgIndexCoordIsUnhealthy(coordID))
|
||||
}
|
||||
|
||||
func msgSegmentNotFound(segID UniqueID) string {
|
||||
return fmt.Sprintf("failed to get segment %d", segID)
|
||||
}
|
||||
|
||||
func errSegmentNotFound(segID UniqueID) error {
|
||||
return fmt.Errorf("%w %d", ErrSegmentNotFound, segID)
|
||||
}
|
||||
|
|
|
@ -17,8 +17,11 @@
|
|||
package indexcoord
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
|
@ -48,3 +51,11 @@ func TestErrIndexCoordIsUnhealthy(t *testing.T) {
|
|||
log.Info("TestErrIndexCoordIsUnhealthy", zap.Error(errIndexCoordIsUnhealthy(nodeID)))
|
||||
}
|
||||
}
|
||||
|
||||
func TestErrSegmentNotExist(t *testing.T) {
|
||||
segID := UniqueID(435846569243121068)
|
||||
errMsg := msgSegmentNotFound(segID)
|
||||
log.Info("TestErrSegmentNotExist", zap.String("errMsg", errMsg))
|
||||
err := errSegmentNotFound(segID)
|
||||
assert.True(t, errors.Is(err, ErrSegmentNotFound))
|
||||
}
|
||||
|
|
|
@ -18,7 +18,10 @@ package indexcoord
|
|||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
|
@ -79,7 +82,7 @@ func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable,
|
|||
wg: sync.WaitGroup{},
|
||||
internalTaskMutex: sync.RWMutex{},
|
||||
childrenTaskMutex: sync.RWMutex{},
|
||||
scheduleDuration: time.Second * 10,
|
||||
scheduleDuration: time.Second,
|
||||
internalNotify: make(chan struct{}, 1),
|
||||
childrenNotify: make(chan struct{}, 1),
|
||||
meta: meta,
|
||||
|
@ -103,12 +106,12 @@ func (fsw *flushedSegmentWatcher) reloadFromKV() error {
|
|||
return err
|
||||
}
|
||||
for _, value := range values {
|
||||
segmentInfo := &datapb.SegmentInfo{}
|
||||
if err = proto.Unmarshal([]byte(value), segmentInfo); err != nil {
|
||||
log.Error("flushSegmentWatcher unmarshal segment info fail", zap.Error(err))
|
||||
segID, err := strconv.ParseInt(value, 10, 64)
|
||||
if err != nil {
|
||||
log.Error("flushSegmentWatcher parse segmentID fail", zap.String("value", value), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
fsw.enqueueInternalTask(segmentInfo)
|
||||
fsw.enqueueInternalTask(segID)
|
||||
}
|
||||
fsw.etcdRevision = version
|
||||
return nil
|
||||
|
@ -124,23 +127,20 @@ func (fsw *flushedSegmentWatcher) Start() {
|
|||
|
||||
func (fsw *flushedSegmentWatcher) Stop() {
|
||||
fsw.cancel()
|
||||
close(fsw.internalNotify)
|
||||
close(fsw.childrenNotify)
|
||||
fsw.wg.Wait()
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) enqueueInternalTask(segmentInfo *datapb.SegmentInfo) {
|
||||
func (fsw *flushedSegmentWatcher) enqueueInternalTask(segmentID UniqueID) {
|
||||
fsw.internalTaskMutex.Lock()
|
||||
defer fsw.internalTaskMutex.Unlock()
|
||||
|
||||
fsw.internalTasks[segmentInfo.ID] = &internalTask{
|
||||
segmentInfo: segmentInfo,
|
||||
fsw.internalTasks[segmentID] = &internalTask{
|
||||
state: indexTaskInit,
|
||||
segmentInfo: nil,
|
||||
}
|
||||
select {
|
||||
case fsw.internalNotify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
fsw.internalTaskMutex.Unlock()
|
||||
|
||||
fsw.prepare(segmentID)
|
||||
fsw.internalNotifyFunc()
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.SegmentInfo, index *model.Index) {
|
||||
|
@ -172,7 +172,6 @@ func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.Segmen
|
|||
switch state.state {
|
||||
case commonpb.IndexState_IndexStateNone:
|
||||
fsw.childrenTasks[segmentInfo.ID][index.IndexID].state = indexTaskInit
|
||||
|
||||
case commonpb.IndexState_InProgress, commonpb.IndexState_Unissued, commonpb.IndexState_Retry:
|
||||
fsw.childrenTasks[segmentInfo.ID][index.IndexID].state = indexTaskInProgress
|
||||
|
||||
|
@ -182,10 +181,7 @@ func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.Segmen
|
|||
// can not to here
|
||||
}
|
||||
}
|
||||
select {
|
||||
case fsw.childrenNotify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
fsw.childrenNotifyFunc()
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) internalScheduler() {
|
||||
|
@ -229,76 +225,164 @@ func (fsw *flushedSegmentWatcher) childrenScheduler() {
|
|||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) internalRun() {
|
||||
fsw.internalTaskMutex.Lock()
|
||||
defer fsw.internalTaskMutex.Unlock()
|
||||
fsw.internalTaskMutex.RLock()
|
||||
segmentIDs := make([]UniqueID, 0, len(fsw.internalTasks))
|
||||
if len(fsw.internalTasks) > 0 {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher schedule internal tasks", zap.Int("internal task num", len(fsw.internalTasks)))
|
||||
for _, t := range fsw.internalTasks {
|
||||
fsw.internalProcess(t)
|
||||
for segID := range fsw.internalTasks {
|
||||
segmentIDs = append(segmentIDs, segID)
|
||||
}
|
||||
sort.Slice(segmentIDs, func(i, j int) bool {
|
||||
return segmentIDs[i] < segmentIDs[j]
|
||||
})
|
||||
}
|
||||
fsw.internalTaskMutex.RUnlock()
|
||||
|
||||
for _, segID := range segmentIDs {
|
||||
fsw.internalProcess(segID)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) childrenRun() {
|
||||
fsw.childrenTaskMutex.Lock()
|
||||
defer fsw.childrenTaskMutex.Unlock()
|
||||
fsw.childrenTaskMutex.RLock()
|
||||
segmentIDs := make([]UniqueID, 0, len(fsw.childrenTasks))
|
||||
if len(fsw.childrenTasks) > 0 {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher schedule children tasks", zap.Int("children task num", len(fsw.childrenTasks)))
|
||||
for segID, tasks := range fsw.childrenTasks {
|
||||
for _, t := range tasks {
|
||||
fsw.childrenProcess(t)
|
||||
}
|
||||
if len(fsw.childrenTasks[segID]) == 0 {
|
||||
delete(fsw.childrenTasks, segID)
|
||||
}
|
||||
for segID := range fsw.childrenTasks {
|
||||
segmentIDs = append(segmentIDs, segID)
|
||||
}
|
||||
sort.Slice(segmentIDs, func(i, j int) bool {
|
||||
return segmentIDs[i] < segmentIDs[j]
|
||||
})
|
||||
}
|
||||
fsw.childrenTaskMutex.RUnlock()
|
||||
for _, segID := range segmentIDs {
|
||||
tasks := fsw.getChildrenTasks(segID)
|
||||
for _, t := range tasks {
|
||||
fsw.childrenProcess(t)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) removeCompactedTasks(t *internalTask) {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher mark task as deleted which is compacted", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
|
||||
fsw.builder.markTasksAsDeleted(fsw.meta.GetBuildIDsFromSegIDs(t.segmentInfo.CompactionFrom))
|
||||
for _, segID := range t.segmentInfo.CompactionFrom {
|
||||
if _, ok := fsw.internalTasks[segID]; ok {
|
||||
fsw.internalTasks[segID].state = indexTaskDeleted
|
||||
func (fsw *flushedSegmentWatcher) internalNotifyFunc() {
|
||||
select {
|
||||
case fsw.internalNotify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) childrenNotifyFunc() {
|
||||
select {
|
||||
case fsw.childrenNotify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) updateInternalTaskState(segID UniqueID, state indexTaskState) {
|
||||
fsw.internalTaskMutex.Lock()
|
||||
defer fsw.internalTaskMutex.Unlock()
|
||||
if _, ok := fsw.internalTasks[segID]; ok {
|
||||
fsw.internalTasks[segID].state = state
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) deleteInternalTask(segID UniqueID) {
|
||||
fsw.internalTaskMutex.Lock()
|
||||
defer fsw.internalTaskMutex.Unlock()
|
||||
|
||||
delete(fsw.internalTasks, segID)
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) getInternalTask(segID UniqueID) *internalTask {
|
||||
fsw.internalTaskMutex.RLock()
|
||||
defer fsw.internalTaskMutex.RUnlock()
|
||||
|
||||
return &internalTask{
|
||||
state: fsw.internalTasks[segID].state,
|
||||
segmentInfo: fsw.internalTasks[segID].segmentInfo,
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) setInternalTaskSegmentInfo(segID UniqueID, segInfo *datapb.SegmentInfo) {
|
||||
fsw.internalTaskMutex.Lock()
|
||||
defer fsw.internalTaskMutex.Unlock()
|
||||
|
||||
if _, ok := fsw.internalTasks[segID]; ok {
|
||||
fsw.internalTasks[segID].segmentInfo = segInfo
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) updateChildrenTaskState(segID, indexID UniqueID, state indexTaskState) {
|
||||
fsw.childrenTaskMutex.Lock()
|
||||
defer fsw.childrenTaskMutex.Unlock()
|
||||
|
||||
if tasks, ok := fsw.childrenTasks[segID]; ok {
|
||||
if _, ok = tasks[indexID]; ok {
|
||||
fsw.childrenTasks[segID][indexID].state = state
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) internalProcess(t *internalTask) {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher process internal task", zap.Int64("segID", t.segmentInfo.ID),
|
||||
func (fsw *flushedSegmentWatcher) hasChildrenTaskDone(segID UniqueID) bool {
|
||||
fsw.childrenTaskMutex.RLock()
|
||||
defer fsw.childrenTaskMutex.RUnlock()
|
||||
if tasks, ok := fsw.childrenTasks[segID]; !ok || len(tasks) == 0 {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) getChildrenTasks(segID UniqueID) map[UniqueID]*childrenTask {
|
||||
fsw.childrenTaskMutex.RLock()
|
||||
defer fsw.childrenTaskMutex.RUnlock()
|
||||
tasks := make(map[UniqueID]*childrenTask)
|
||||
if ts, ok := fsw.childrenTasks[segID]; ok {
|
||||
for k, v := range ts {
|
||||
tasks[k] = v
|
||||
}
|
||||
}
|
||||
return tasks
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) deleteChildTask(segID, indexID UniqueID) {
|
||||
fsw.childrenTaskMutex.Lock()
|
||||
defer fsw.childrenTaskMutex.Unlock()
|
||||
if _, ok := fsw.childrenTasks[segID]; ok {
|
||||
delete(fsw.childrenTasks[segID], indexID)
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) deleteChildrenTask(segID UniqueID) {
|
||||
fsw.childrenTaskMutex.Lock()
|
||||
defer fsw.childrenTaskMutex.Unlock()
|
||||
|
||||
delete(fsw.childrenTasks, segID)
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) internalProcess(segID UniqueID) {
|
||||
// first pull segmentInfo
|
||||
if err := fsw.pullSegmentInfo(segID); err != nil {
|
||||
return
|
||||
}
|
||||
t := fsw.getInternalTask(segID)
|
||||
log.Debug("IndexCoord flushedSegmentWatcher process internal task", zap.Int64("segID", segID),
|
||||
zap.String("state", t.state.String()))
|
||||
|
||||
switch t.state {
|
||||
case indexTaskInit:
|
||||
if t.segmentInfo.CreatedByCompaction {
|
||||
fsw.removeCompactedTasks(t)
|
||||
}
|
||||
if err := fsw.constructTask(t); err != nil {
|
||||
log.Error("IndexCoord flushedSegmentWatcher construct task fail", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.Int64s("compactFrom", t.segmentInfo.CompactionFrom), zap.Error(err))
|
||||
return
|
||||
}
|
||||
fsw.internalTasks[t.segmentInfo.ID].state = indexTaskInProgress
|
||||
select {
|
||||
case fsw.internalNotify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
return
|
||||
fsw.constructTask(t)
|
||||
fsw.updateInternalTaskState(segID, indexTaskInProgress)
|
||||
fsw.internalNotifyFunc()
|
||||
case indexTaskInProgress:
|
||||
fsw.childrenTaskMutex.RLock()
|
||||
defer fsw.childrenTaskMutex.RUnlock()
|
||||
if tasks, ok := fsw.childrenTasks[t.segmentInfo.ID]; !ok || len(tasks) == 0 {
|
||||
fsw.internalTasks[t.segmentInfo.ID].state = indexTaskDone
|
||||
select {
|
||||
case fsw.internalNotify <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
if fsw.hasChildrenTaskDone(segID) {
|
||||
fsw.updateInternalTaskState(segID, indexTaskDone)
|
||||
fsw.internalNotifyFunc()
|
||||
}
|
||||
return
|
||||
case indexTaskDone:
|
||||
handoffTask := &querypb.SegmentInfo{
|
||||
SegmentID: t.segmentInfo.ID,
|
||||
SegmentID: segID,
|
||||
CollectionID: t.segmentInfo.CollectionID,
|
||||
PartitionID: t.segmentInfo.PartitionID,
|
||||
NumRows: t.segmentInfo.NumOfRows,
|
||||
|
@ -313,25 +397,21 @@ func (fsw *flushedSegmentWatcher) internalProcess(t *internalTask) {
|
|||
}
|
||||
if err := fsw.writeHandoffSegment(handoffTask); err != nil {
|
||||
log.Error("IndexCoord flushSegmentWatcher writeHandoffSegment with no index fail",
|
||||
zap.Int64("segID", t.segmentInfo.ID), zap.Error(err))
|
||||
zap.Int64("segID", segID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
if err := fsw.removeFlushedSegment(t); err != nil {
|
||||
log.Error("IndexCoord flushSegmentWatcher removeFlushedSegment fail",
|
||||
zap.Int64("segID", t.segmentInfo.ID), zap.Error(err))
|
||||
zap.Int64("segID", segID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
delete(fsw.internalTasks, t.segmentInfo.ID)
|
||||
return
|
||||
fsw.deleteInternalTask(segID)
|
||||
fsw.internalNotifyFunc()
|
||||
case indexTaskDeleted:
|
||||
if t.segmentInfo.CreatedByCompaction {
|
||||
fsw.removeCompactedTasks(t)
|
||||
}
|
||||
fsw.childrenTaskMutex.Lock()
|
||||
delete(fsw.childrenTasks, t.segmentInfo.ID)
|
||||
fsw.childrenTaskMutex.Unlock()
|
||||
fsw.internalTasks[t.segmentInfo.ID].state = indexTaskDone
|
||||
return
|
||||
fsw.updateInternalTaskState(segID, indexTaskDone)
|
||||
default:
|
||||
log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.String("state", t.state.String()))
|
||||
|
@ -375,26 +455,34 @@ func (fsw *flushedSegmentWatcher) childrenProcess(task *childrenTask) {
|
|||
fsw.builder.enqueue(buildID)
|
||||
}
|
||||
|
||||
fsw.childrenTasks[segID][task.indexInfo.IndexID].state = indexTaskInProgress
|
||||
return
|
||||
fsw.updateChildrenTaskState(segID, task.indexInfo.IndexID, indexTaskInProgress)
|
||||
fsw.childrenNotifyFunc()
|
||||
case indexTaskInProgress:
|
||||
filePath, err := fsw.meta.GetIndexFilePathInfo(segID, task.indexInfo.IndexID)
|
||||
if err != nil {
|
||||
log.Warn("IndexCoord get index file path fail", zap.Int64("collID", task.segmentInfo.CollectionID),
|
||||
zap.Int64("partID", task.segmentInfo.PartitionID), zap.Int64("segID", segID), zap.Error(err))
|
||||
state := fsw.meta.GetSegmentIndexState(task.segmentInfo.ID, task.indexInfo.IndexID)
|
||||
if state.state == commonpb.IndexState_IndexStateNone {
|
||||
log.Debug("task is no need to build index, remove task", zap.Int64("segID", task.segmentInfo.ID),
|
||||
zap.Int64("indexID", task.indexInfo.IndexID))
|
||||
fsw.deleteChildTask(task.segmentInfo.ID, task.indexInfo.IndexID)
|
||||
fsw.childrenNotifyFunc()
|
||||
return
|
||||
}
|
||||
fsw.childrenTasks[segID][task.indexInfo.IndexID].indexInfo.IndexFilePaths = filePath.IndexFilePaths
|
||||
fsw.childrenTasks[segID][task.indexInfo.IndexID].indexInfo.IndexSize = int64(filePath.SerializedSize)
|
||||
fsw.childrenTasks[segID][task.indexInfo.IndexID].state = indexTaskDone
|
||||
|
||||
return
|
||||
if state.state != commonpb.IndexState_Finished && state.state != commonpb.IndexState_Failed {
|
||||
log.Debug("the index on segment is not finish", zap.Int64("segID", segID),
|
||||
zap.String("state", state.state.String()), zap.String("fail reason", state.failReason))
|
||||
return
|
||||
}
|
||||
// don't set index files, QueryCoord get index files from IndexCoord by grpc.
|
||||
//fsw.childrenTasks[segID][task.indexInfo.IndexID].indexInfo.IndexFilePaths = filePath.IndexFilePaths
|
||||
//fsw.childrenTasks[segID][task.indexInfo.IndexID].indexInfo.IndexSize = int64(filePath.SerializedSize)
|
||||
fsw.updateChildrenTaskState(segID, task.indexInfo.IndexID, indexTaskDone)
|
||||
fsw.childrenNotifyFunc()
|
||||
case indexTaskDone:
|
||||
handoffTask := &querypb.SegmentInfo{
|
||||
SegmentID: task.segmentInfo.ID,
|
||||
CollectionID: task.segmentInfo.CollectionID,
|
||||
PartitionID: task.segmentInfo.PartitionID,
|
||||
NumRows: task.segmentInfo.NumOfRows,
|
||||
DmChannel: task.segmentInfo.GetInsertChannel(),
|
||||
IndexName: task.indexInfo.IndexName,
|
||||
IndexID: task.indexInfo.IndexID,
|
||||
CompactionFrom: task.segmentInfo.CompactionFrom,
|
||||
|
@ -408,31 +496,30 @@ func (fsw *flushedSegmentWatcher) childrenProcess(task *childrenTask) {
|
|||
zap.Int64("partID", task.segmentInfo.PartitionID), zap.Int64("segID", task.segmentInfo.ID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
log.Warn("IndexCoord writeHandoffSegment success", zap.Int64("collID", task.segmentInfo.CollectionID),
|
||||
log.Debug("IndexCoord writeHandoffSegment success", zap.Int64("collID", task.segmentInfo.CollectionID),
|
||||
zap.Int64("partID", task.segmentInfo.PartitionID), zap.Int64("segID", task.segmentInfo.ID))
|
||||
delete(fsw.childrenTasks[task.segmentInfo.ID], task.indexInfo.IndexID)
|
||||
return
|
||||
fsw.deleteChildTask(task.segmentInfo.ID, task.indexInfo.IndexID)
|
||||
fsw.childrenNotifyFunc()
|
||||
default:
|
||||
log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", task.segmentInfo.ID),
|
||||
zap.String("state", task.state.String()))
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) constructTask(t *internalTask) error {
|
||||
func (fsw *flushedSegmentWatcher) constructTask(t *internalTask) {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher construct tasks by segment info", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
|
||||
fieldIndexes := fsw.meta.GetIndexesForCollection(t.segmentInfo.CollectionID, "")
|
||||
if t.segmentInfo.NumOfRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex || len(fieldIndexes) == 0 {
|
||||
if len(fieldIndexes) == 0 {
|
||||
log.Debug("segment no need to build index", zap.Int64("segmentID", t.segmentInfo.ID),
|
||||
zap.Int64("num of rows", t.segmentInfo.NumOfRows), zap.Int("collection indexes num", len(fieldIndexes)))
|
||||
// no need to build index
|
||||
return nil
|
||||
return
|
||||
}
|
||||
|
||||
for _, index := range fieldIndexes {
|
||||
fsw.enqueueChildrenTask(t.segmentInfo, index)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) writeHandoffSegment(t *querypb.SegmentInfo) error {
|
||||
|
@ -456,16 +543,6 @@ func (fsw *flushedSegmentWatcher) writeHandoffSegment(t *querypb.SegmentInfo) er
|
|||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) removeFlushedSegment(t *internalTask) error {
|
||||
if t.segmentInfo.CreatedByCompaction {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher mark the segments indexes as deleted which is compacted", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
|
||||
if err := fsw.meta.MarkSegmentsIndexAsDeleted(t.segmentInfo.CompactionFrom); err != nil {
|
||||
log.Error("IndexCoord mark compacted segments' index fail", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.Int64s("compactFrom", t.segmentInfo.CompactionFrom), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
deletedKeys := fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, t.segmentInfo.CollectionID, t.segmentInfo.PartitionID, t.segmentInfo.ID)
|
||||
err := fsw.kvClient.RemoveWithPrefix(deletedKeys)
|
||||
if err != nil {
|
||||
|
@ -477,3 +554,63 @@ func (fsw *flushedSegmentWatcher) removeFlushedSegment(t *internalTask) error {
|
|||
zap.Int64("partID", t.segmentInfo.PartitionID), zap.Int64("segID", t.segmentInfo.ID))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) pullSegmentInfo(segmentID UniqueID) error {
|
||||
t := fsw.getInternalTask(segmentID)
|
||||
if t.segmentInfo != nil {
|
||||
return nil
|
||||
}
|
||||
resp, err := fsw.ic.dataCoordClient.GetSegmentInfo(fsw.ctx, &datapb.GetSegmentInfoRequest{
|
||||
SegmentIDs: []int64{segmentID},
|
||||
IncludeUnHealthy: true,
|
||||
})
|
||||
if err != nil {
|
||||
log.Error("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segmentID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
if resp.Status.GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Error("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segmentID),
|
||||
zap.String("fail reason", resp.Status.GetReason()))
|
||||
if resp.Status.GetReason() == msgSegmentNotFound(segmentID) {
|
||||
return errSegmentNotFound(segmentID)
|
||||
}
|
||||
return errors.New(resp.Status.GetReason())
|
||||
}
|
||||
for _, info := range resp.Infos {
|
||||
if info.ID == segmentID {
|
||||
fsw.setInternalTaskSegmentInfo(segmentID, info)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
errMsg := fmt.Sprintf("flushedSegmentWatcher get segment info fail, the segment is not include in the response with ID: %d", segmentID)
|
||||
log.Error(errMsg)
|
||||
return errors.New(errMsg)
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) prepare(segID UniqueID) {
|
||||
if err := fsw.pullSegmentInfo(segID); err != nil {
|
||||
log.Error("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segID),
|
||||
zap.Error(err))
|
||||
if errors.Is(err, ErrSegmentNotFound) {
|
||||
fsw.deleteInternalTask(segID)
|
||||
return
|
||||
}
|
||||
return
|
||||
}
|
||||
t := fsw.getInternalTask(segID)
|
||||
if t.segmentInfo.CreatedByCompaction {
|
||||
fsw.removeCompactedTasks(t)
|
||||
}
|
||||
}
|
||||
|
||||
func (fsw *flushedSegmentWatcher) removeCompactedTasks(t *internalTask) {
|
||||
log.Debug("IndexCoord flushedSegmentWatcher mark task as deleted which is compacted", zap.Int64("segID", t.segmentInfo.ID),
|
||||
zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
|
||||
fsw.builder.markTasksAsDeleted(fsw.meta.GetBuildIDsFromSegIDs(t.segmentInfo.CompactionFrom))
|
||||
for _, segID := range t.segmentInfo.CompactionFrom {
|
||||
fsw.deleteChildrenTask(segID)
|
||||
if _, ok := fsw.internalTasks[segID]; ok {
|
||||
fsw.updateInternalTaskState(segID, indexTaskDeleted)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,55 +1,392 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package indexcoord
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
)
|
||||
|
||||
func Test_flushSegmentWatcher(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
segInfo := &datapb.SegmentInfo{
|
||||
ID: 1,
|
||||
CollectionID: 2,
|
||||
PartitionID: 3,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 1024,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
MaxRowNum: 1024,
|
||||
LastExpireTime: 0,
|
||||
StartPosition: &internalpb.MsgPosition{
|
||||
Timestamp: createTs,
|
||||
},
|
||||
DmlPosition: nil,
|
||||
Binlogs: nil,
|
||||
Statslogs: nil,
|
||||
Deltalogs: nil,
|
||||
CreatedByCompaction: false,
|
||||
CompactionFrom: nil,
|
||||
DroppedAt: 0,
|
||||
}
|
||||
value, err := proto.Marshal(segInfo)
|
||||
assert.Nil(t, err)
|
||||
|
||||
watcher, err := newFlushSegmentWatcher(ctx, &mockETCDKV{
|
||||
loadWithRevision: func(key string) ([]string, []string, int64, error) {
|
||||
return []string{"seg1"}, []string{string(value)}, 1, nil
|
||||
watcher, err := newFlushSegmentWatcher(ctx,
|
||||
&mockETCDKV{
|
||||
loadWithRevision: func(key string) ([]string, []string, int64, error) {
|
||||
return []string{"seg1"}, []string{"12345"}, 1, nil
|
||||
},
|
||||
},
|
||||
},
|
||||
&metaTable{
|
||||
catalog: &indexcoord.Catalog{
|
||||
Txn: NewMockEtcdKV(),
|
||||
},
|
||||
},
|
||||
&indexBuilder{}, &IndexCoord{})
|
||||
&indexBuilder{}, &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, watcher)
|
||||
}
|
||||
|
||||
func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) {
|
||||
t.Run("success", func(t *testing.T) {
|
||||
fsw, err := newFlushSegmentWatcher(context.Background(),
|
||||
&mockETCDKV{
|
||||
loadWithRevision: func(key string) ([]string, []string, int64, error) {
|
||||
return []string{"segID1"}, []string{"12345"}, 1, nil
|
||||
},
|
||||
}, &metaTable{}, &indexBuilder{}, &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(fsw.internalTasks))
|
||||
})
|
||||
|
||||
t.Run("load fail", func(t *testing.T) {
|
||||
fsw, err := newFlushSegmentWatcher(context.Background(),
|
||||
&mockETCDKV{
|
||||
loadWithRevision: func(key string) ([]string, []string, int64, error) {
|
||||
return []string{"segID1"}, []string{"12345"}, 1, errors.New("error")
|
||||
},
|
||||
}, &metaTable{}, &indexBuilder{}, &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, fsw)
|
||||
})
|
||||
|
||||
t.Run("parse fail", func(t *testing.T) {
|
||||
fsw, err := newFlushSegmentWatcher(context.Background(),
|
||||
&mockETCDKV{
|
||||
loadWithRevision: func(key string) ([]string, []string, int64, error) {
|
||||
return []string{"segID1"}, []string{"segID"}, 1, nil
|
||||
},
|
||||
}, &metaTable{}, &indexBuilder{}, &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, fsw)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_flushSegmentWatcher_prepare(t *testing.T) {
|
||||
task := &internalTask{
|
||||
state: indexTaskInit,
|
||||
segmentInfo: nil,
|
||||
}
|
||||
t.Run("success", func(t *testing.T) {
|
||||
fsw := &flushedSegmentWatcher{
|
||||
ic: &IndexCoord{
|
||||
dataCoordClient: &DataCoordMock{
|
||||
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return &datapb.GetSegmentInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
Infos: []*datapb.SegmentInfo{
|
||||
{
|
||||
ID: segID + 100,
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
NumOfRows: 10000,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
CreatedByCompaction: true,
|
||||
CompactionFrom: []int64{segID},
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
},
|
||||
},
|
||||
},
|
||||
internalTasks: map[UniqueID]*internalTask{
|
||||
segID: task,
|
||||
segID + 100: {state: indexTaskInit, segmentInfo: nil},
|
||||
},
|
||||
meta: &metaTable{
|
||||
segmentIndexLock: sync.RWMutex{},
|
||||
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
|
||||
segID: {
|
||||
indexID: {
|
||||
SegmentID: segID,
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
NumRows: 1000,
|
||||
IndexID: indexID,
|
||||
BuildID: buildID,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
builder: &indexBuilder{
|
||||
taskMutex: sync.RWMutex{},
|
||||
scheduleDuration: 0,
|
||||
tasks: nil,
|
||||
notifyChan: nil,
|
||||
},
|
||||
}
|
||||
fsw.prepare(segID + 100)
|
||||
// idempotent
|
||||
fsw.prepare(segID + 100)
|
||||
})
|
||||
t.Run("init task get segmentInfo fail", func(t *testing.T) {
|
||||
fsw := &flushedSegmentWatcher{
|
||||
ic: &IndexCoord{
|
||||
dataCoordClient: &DataCoordMock{
|
||||
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return nil, errors.New("error")
|
||||
},
|
||||
},
|
||||
},
|
||||
internalTasks: map[UniqueID]*internalTask{
|
||||
segID: task,
|
||||
},
|
||||
}
|
||||
fsw.prepare(segID)
|
||||
fsw.internalProcess(segID)
|
||||
fsw.ic.dataCoordClient = &DataCoordMock{
|
||||
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return &datapb.GetSegmentInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "fail reason",
|
||||
},
|
||||
}, nil
|
||||
},
|
||||
}
|
||||
fsw.prepare(segID)
|
||||
|
||||
fsw.ic.dataCoordClient = &DataCoordMock{
|
||||
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return &datapb.GetSegmentInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: msgSegmentNotFound(segID),
|
||||
},
|
||||
}, nil
|
||||
},
|
||||
}
|
||||
fsw.prepare(segID)
|
||||
_, ok := fsw.internalTasks[segID]
|
||||
assert.False(t, ok)
|
||||
|
||||
fsw.ic.dataCoordClient = &DataCoordMock{
|
||||
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return &datapb.GetSegmentInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
Infos: []*datapb.SegmentInfo{
|
||||
{
|
||||
ID: segID + 100,
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 10000,
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
},
|
||||
}
|
||||
fsw.internalTasks = map[UniqueID]*internalTask{
|
||||
segID: task,
|
||||
}
|
||||
fsw.prepare(segID)
|
||||
})
|
||||
|
||||
t.Run("done task write handoff event fail", func(t *testing.T) {
|
||||
task := &internalTask{
|
||||
state: indexTaskDone,
|
||||
segmentInfo: &datapb.SegmentInfo{
|
||||
ID: 0,
|
||||
CollectionID: 0,
|
||||
PartitionID: 0,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 0,
|
||||
State: 0,
|
||||
MaxRowNum: 0,
|
||||
LastExpireTime: 0,
|
||||
StartPosition: nil,
|
||||
DmlPosition: nil,
|
||||
Binlogs: nil,
|
||||
Statslogs: nil,
|
||||
Deltalogs: nil,
|
||||
CreatedByCompaction: false,
|
||||
CompactionFrom: nil,
|
||||
DroppedAt: 0,
|
||||
},
|
||||
}
|
||||
fsw := &flushedSegmentWatcher{
|
||||
ic: &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
},
|
||||
kvClient: &mockETCDKV{
|
||||
save: func(s string, s2 string) error {
|
||||
return errors.New("error")
|
||||
},
|
||||
},
|
||||
internalTasks: map[UniqueID]*internalTask{
|
||||
segID: task,
|
||||
},
|
||||
}
|
||||
fsw.prepare(segID)
|
||||
fsw.internalProcess(segID)
|
||||
})
|
||||
|
||||
t.Run("done task remove flush segment fail", func(t *testing.T) {
|
||||
task := &internalTask{
|
||||
state: indexTaskDone,
|
||||
segmentInfo: &datapb.SegmentInfo{
|
||||
ID: 0,
|
||||
CollectionID: 0,
|
||||
PartitionID: 0,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 0,
|
||||
State: 0,
|
||||
MaxRowNum: 0,
|
||||
LastExpireTime: 0,
|
||||
StartPosition: nil,
|
||||
DmlPosition: nil,
|
||||
Binlogs: nil,
|
||||
Statslogs: nil,
|
||||
Deltalogs: nil,
|
||||
CreatedByCompaction: false,
|
||||
CompactionFrom: nil,
|
||||
DroppedAt: 0,
|
||||
},
|
||||
}
|
||||
fsw := &flushedSegmentWatcher{
|
||||
ic: &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
},
|
||||
kvClient: &mockETCDKV{
|
||||
save: func(s string, s2 string) error {
|
||||
return nil
|
||||
},
|
||||
removeWithPrefix: func(s string) error {
|
||||
return errors.New("error")
|
||||
},
|
||||
},
|
||||
internalTasks: map[UniqueID]*internalTask{
|
||||
segID: task,
|
||||
},
|
||||
}
|
||||
|
||||
fsw.prepare(segID)
|
||||
fsw.internalProcess(segID)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_flushSegmentWatcher_childrenProcess_error(t *testing.T) {
|
||||
task := &childrenTask{
|
||||
internalTask: internalTask{
|
||||
state: indexTaskInProgress,
|
||||
segmentInfo: &datapb.SegmentInfo{
|
||||
ID: segID,
|
||||
CollectionID: 0,
|
||||
PartitionID: 0,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 0,
|
||||
State: 0,
|
||||
MaxRowNum: 0,
|
||||
LastExpireTime: 0,
|
||||
StartPosition: &internalpb.MsgPosition{
|
||||
Timestamp: 1,
|
||||
},
|
||||
},
|
||||
},
|
||||
indexInfo: &querypb.FieldIndexInfo{
|
||||
FieldID: 0,
|
||||
EnableIndex: true,
|
||||
IndexName: "",
|
||||
IndexID: indexID,
|
||||
BuildID: buildID,
|
||||
IndexParams: nil,
|
||||
IndexFilePaths: nil,
|
||||
IndexSize: 0,
|
||||
},
|
||||
}
|
||||
|
||||
t.Run("inProgress task not finish", func(t *testing.T) {
|
||||
fsw := &flushedSegmentWatcher{
|
||||
ic: &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
},
|
||||
childrenTasks: map[UniqueID]map[UniqueID]*childrenTask{
|
||||
segID: {
|
||||
indexID: task,
|
||||
},
|
||||
},
|
||||
meta: &metaTable{
|
||||
segmentIndexLock: sync.RWMutex{},
|
||||
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
|
||||
segID: {
|
||||
indexID: &model.SegmentIndex{
|
||||
SegmentID: segID,
|
||||
CollectionID: 0,
|
||||
PartitionID: 0,
|
||||
NumRows: 0,
|
||||
IndexID: indexID,
|
||||
BuildID: buildID,
|
||||
NodeID: 1,
|
||||
IndexVersion: 1,
|
||||
IndexState: commonpb.IndexState_InProgress,
|
||||
FailReason: "",
|
||||
IsDeleted: false,
|
||||
CreateTime: 0,
|
||||
IndexFilePaths: nil,
|
||||
IndexSize: 0,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
fsw.childrenProcess(task)
|
||||
})
|
||||
|
||||
t.Run("inProgress not in meta", func(t *testing.T) {
|
||||
fsw := &flushedSegmentWatcher{
|
||||
ic: &IndexCoord{
|
||||
dataCoordClient: NewDataCoordMock(),
|
||||
},
|
||||
childrenTasks: map[UniqueID]map[UniqueID]*childrenTask{
|
||||
segID: {
|
||||
indexID: task,
|
||||
},
|
||||
},
|
||||
meta: &metaTable{
|
||||
segmentIndexLock: sync.RWMutex{},
|
||||
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{},
|
||||
},
|
||||
}
|
||||
|
||||
fsw.childrenProcess(task)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -19,17 +19,16 @@ package indexcoord
|
|||
import (
|
||||
"context"
|
||||
"path"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
)
|
||||
|
||||
|
@ -42,8 +41,8 @@ type indexBuilder struct {
|
|||
scheduleDuration time.Duration
|
||||
|
||||
// TODO @xiaocai2333: use priority queue
|
||||
tasks map[int64]indexTaskState
|
||||
notify chan struct{}
|
||||
tasks map[int64]indexTaskState
|
||||
notifyChan chan struct{}
|
||||
|
||||
ic *IndexCoord
|
||||
|
||||
|
@ -58,9 +57,9 @@ func newIndexBuilder(ctx context.Context, ic *IndexCoord, metaTable *metaTable,
|
|||
cancel: cancel,
|
||||
meta: metaTable,
|
||||
ic: ic,
|
||||
tasks: make(map[int64]indexTaskState, 1024),
|
||||
notify: make(chan struct{}, 1),
|
||||
scheduleDuration: time.Second,
|
||||
tasks: make(map[int64]indexTaskState),
|
||||
notifyChan: make(chan struct{}, 1),
|
||||
scheduleDuration: time.Second * 3,
|
||||
}
|
||||
ib.reloadFromKV(aliveNodes)
|
||||
return ib
|
||||
|
@ -73,7 +72,6 @@ func (ib *indexBuilder) Start() {
|
|||
|
||||
func (ib *indexBuilder) Stop() {
|
||||
ib.cancel()
|
||||
close(ib.notify)
|
||||
ib.wg.Wait()
|
||||
}
|
||||
|
||||
|
@ -118,19 +116,24 @@ func (ib *indexBuilder) reloadFromKV(aliveNodes []UniqueID) {
|
|||
}
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) enqueue(buildID UniqueID) {
|
||||
// notify
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
ib.tasks[buildID] = indexTaskInit
|
||||
// notify is an unblocked notify function
|
||||
func (ib *indexBuilder) notify() {
|
||||
select {
|
||||
case ib.notify <- struct{}{}:
|
||||
case ib.notifyChan <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) enqueue(buildID UniqueID) {
|
||||
defer ib.notify()
|
||||
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
ib.tasks[buildID] = indexTaskInit
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) schedule() {
|
||||
// receive notify
|
||||
// receive notifyChan
|
||||
// time ticker
|
||||
log.Info("index builder schedule loop start")
|
||||
defer ib.wg.Done()
|
||||
|
@ -141,43 +144,82 @@ func (ib *indexBuilder) schedule() {
|
|||
case <-ib.ctx.Done():
|
||||
log.Warn("index builder ctx done")
|
||||
return
|
||||
case _, ok := <-ib.notify:
|
||||
case _, ok := <-ib.notifyChan:
|
||||
if ok {
|
||||
ib.taskMutex.Lock()
|
||||
if len(ib.tasks) > 0 {
|
||||
log.Info("index builder task schedule", zap.Int("task num", len(ib.tasks)))
|
||||
for buildID := range ib.tasks {
|
||||
ib.process(buildID)
|
||||
}
|
||||
}
|
||||
ib.taskMutex.Unlock()
|
||||
ib.run()
|
||||
}
|
||||
// !ok means indexBuilder is closed.
|
||||
// !ok means indexBuild is closed.
|
||||
case <-ticker.C:
|
||||
ib.taskMutex.Lock()
|
||||
if len(ib.tasks) > 0 {
|
||||
log.Info("index builder task schedule", zap.Int("task num", len(ib.tasks)))
|
||||
for buildID := range ib.tasks {
|
||||
ib.process(buildID)
|
||||
}
|
||||
}
|
||||
ib.taskMutex.Unlock()
|
||||
ib.run()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) run() {
|
||||
ib.taskMutex.RLock()
|
||||
log.Info("index builder task schedule", zap.Int("task num", len(ib.tasks)))
|
||||
buildIDs := make([]UniqueID, 0, len(ib.tasks))
|
||||
for tID := range ib.tasks {
|
||||
buildIDs = append(buildIDs, tID)
|
||||
}
|
||||
ib.taskMutex.RUnlock()
|
||||
|
||||
sort.Slice(buildIDs, func(i, j int) bool {
|
||||
return buildIDs[i] < buildIDs[j]
|
||||
})
|
||||
for _, buildID := range buildIDs {
|
||||
ib.process(buildID)
|
||||
}
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) process(buildID UniqueID) {
|
||||
defer ib.notify()
|
||||
ib.taskMutex.RLock()
|
||||
state := ib.tasks[buildID]
|
||||
ib.taskMutex.RUnlock()
|
||||
|
||||
updateStateFunc := func(buildID UniqueID, state indexTaskState) {
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
ib.tasks[buildID] = state
|
||||
}
|
||||
|
||||
deleteFunc := func(buildID UniqueID) {
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
delete(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)
|
||||
if !exist {
|
||||
log.Debug("index task has not exist in meta table, remove task", zap.Int64("buildID", buildID))
|
||||
deleteFunc(buildID)
|
||||
return
|
||||
}
|
||||
|
||||
switch state {
|
||||
case indexTaskInit:
|
||||
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
delete(ib.tasks, buildID)
|
||||
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
deleteFunc(buildID)
|
||||
return
|
||||
}
|
||||
log.Debug("task state is init, build index ...", zap.Int64("buildID", buildID),
|
||||
zap.Int64("segID", meta.SegmentID), zap.Int64("num rows", meta.NumRows))
|
||||
if meta.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex {
|
||||
if err := ib.meta.FinishTask(&indexpb.IndexTaskInfo{
|
||||
BuildID: buildID,
|
||||
State: commonpb.IndexState_Finished,
|
||||
IndexFiles: nil,
|
||||
SerializedSize: 0,
|
||||
FailReason: "",
|
||||
}); err != nil {
|
||||
log.Error("IndexCoord update index state fail", zap.Int64("buildID", buildID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
updateStateFunc(buildID, indexTaskDone)
|
||||
return
|
||||
}
|
||||
log.Debug("task state is init, build index ...", zap.Int64("buildID", buildID))
|
||||
// peek client
|
||||
// if all IndexNodes are executing task, wait for one of them to finish the task.
|
||||
nodeID, client := ib.ic.nodeManager.PeekClient(meta)
|
||||
|
@ -195,7 +237,7 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
if err := ib.ic.tryAcquireSegmentReferLock(ib.ctx, buildID, nodeID, []UniqueID{meta.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
|
||||
updateStateFunc(buildID, indexTaskRetry)
|
||||
return
|
||||
}
|
||||
segmentsInfo, err := ib.ic.dataCoordClient.GetSegmentInfo(ib.ctx, &datapb.GetSegmentInfoRequest{
|
||||
|
@ -206,13 +248,17 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
if err != nil {
|
||||
log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID),
|
||||
zap.Int64("buildID", buildID), zap.Error(err))
|
||||
ib.tasks[buildID] = indexTaskRetry
|
||||
updateStateFunc(buildID, indexTaskRetry)
|
||||
return
|
||||
}
|
||||
if segmentsInfo.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID),
|
||||
zap.Int64("buildID", buildID), zap.String("failReason", segmentsInfo.Status.Reason))
|
||||
ib.tasks[buildID] = indexTaskRetry
|
||||
if segmentsInfo.Status.GetReason() == msgSegmentNotFound(meta.SegmentID) {
|
||||
updateStateFunc(buildID, indexTaskDeleted)
|
||||
return
|
||||
}
|
||||
updateStateFunc(buildID, indexTaskRetry)
|
||||
return
|
||||
}
|
||||
binLogs := make([]string, 0)
|
||||
|
@ -232,12 +278,7 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
break
|
||||
}
|
||||
|
||||
typeParams, err := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID)
|
||||
if err != nil {
|
||||
log.Warn("get type params fail", zap.Int64("buildID", buildID), zap.Error(err))
|
||||
ib.tasks[buildID] = indexTaskRetry
|
||||
return
|
||||
}
|
||||
typeParams := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID)
|
||||
indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID)
|
||||
|
||||
var storageConfig *indexpb.StorageConfig
|
||||
|
@ -275,7 +316,7 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
// 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
|
||||
updateStateFunc(buildID, indexTaskRetry)
|
||||
return
|
||||
}
|
||||
// update index meta state to InProgress
|
||||
|
@ -283,16 +324,16 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
// 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
|
||||
updateStateFunc(buildID, indexTaskRetry)
|
||||
return
|
||||
}
|
||||
log.Debug("index task assigned success", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
|
||||
ib.tasks[buildID] = indexTaskInProgress
|
||||
updateStateFunc(buildID, indexTaskInProgress)
|
||||
|
||||
case indexTaskDone:
|
||||
log.Debug("index task has done", zap.Int64("buildID", buildID))
|
||||
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
ib.tasks[buildID] = indexTaskDeleted
|
||||
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
updateStateFunc(buildID, indexTaskDeleted)
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -304,11 +345,11 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
log.Error("index builder try to release reference lock failed", zap.Error(err))
|
||||
return
|
||||
}
|
||||
delete(ib.tasks, buildID)
|
||||
deleteFunc(buildID)
|
||||
case indexTaskRetry:
|
||||
log.Debug("index task state is retry, try to release reference lock", zap.Int64("buildID", buildID))
|
||||
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
ib.tasks[buildID] = indexTaskDeleted
|
||||
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
updateStateFunc(buildID, indexTaskDeleted)
|
||||
return
|
||||
}
|
||||
if err := ib.releaseLockAndResetTask(buildID, meta.NodeID); err != nil {
|
||||
|
@ -316,7 +357,8 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
log.Error("index builder try to release reference lock failed", zap.Error(err))
|
||||
return
|
||||
}
|
||||
ib.tasks[buildID] = indexTaskInit
|
||||
|
||||
updateStateFunc(buildID, indexTaskInit)
|
||||
|
||||
case indexTaskDeleted:
|
||||
log.Debug("index task state is deleted, try to release reference lock", zap.Int64("buildID", buildID))
|
||||
|
@ -332,16 +374,16 @@ func (ib *indexBuilder) process(buildID UniqueID) {
|
|||
}
|
||||
}
|
||||
// reset nodeID success, remove task.
|
||||
delete(ib.tasks, buildID)
|
||||
deleteFunc(buildID)
|
||||
|
||||
default:
|
||||
log.Debug("index task is in progress", zap.Int64("buildID", buildID),
|
||||
zap.String("state", meta.IndexState.String()))
|
||||
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
ib.tasks[buildID] = indexTaskDeleted
|
||||
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
|
||||
updateStateFunc(buildID, indexTaskDeleted)
|
||||
return
|
||||
}
|
||||
ib.tasks[buildID] = ib.getTaskState(buildID, meta.NodeID)
|
||||
updateStateFunc(buildID, ib.getTaskState(buildID, meta.NodeID))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -366,24 +408,29 @@ func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
|
|||
|
||||
// indexInfos length is always one.
|
||||
for _, info := range response.IndexInfos {
|
||||
if info.State == commonpb.IndexState_Failed || info.State == commonpb.IndexState_Finished {
|
||||
log.Info("this task has been finished", zap.Int64("buildID", info.BuildID),
|
||||
zap.String("index state", info.State.String()))
|
||||
if err := ib.meta.FinishTask(info); err != nil {
|
||||
log.Error("IndexCoord update index state fail", zap.Int64("buildID", info.BuildID),
|
||||
zap.String("index state", info.State.String()), zap.Error(err))
|
||||
return indexTaskInProgress
|
||||
if info.BuildID == buildID {
|
||||
if info.State == commonpb.IndexState_Failed || info.State == commonpb.IndexState_Finished {
|
||||
log.Info("this task has been finished", zap.Int64("buildID", info.BuildID),
|
||||
zap.String("index state", info.State.String()))
|
||||
if err := ib.meta.FinishTask(info); err != nil {
|
||||
log.Error("IndexCoord update index state fail", zap.Int64("buildID", info.BuildID),
|
||||
zap.String("index state", info.State.String()), zap.Error(err))
|
||||
return indexTaskInProgress
|
||||
}
|
||||
return indexTaskDone
|
||||
} else if info.State == commonpb.IndexState_Retry || info.State == commonpb.IndexState_IndexStateNone {
|
||||
log.Info("this task should be retry", zap.Int64("buildID", buildID))
|
||||
return indexTaskRetry
|
||||
}
|
||||
return indexTaskDone
|
||||
} else if info.State == commonpb.IndexState_Retry {
|
||||
log.Info("this task should be retry", zap.Int64("buildID", buildID))
|
||||
return indexTaskRetry
|
||||
return indexTaskInProgress
|
||||
}
|
||||
return indexTaskInProgress
|
||||
}
|
||||
log.Info("this task should be retry, indexNode does not have this task", zap.Int64("buildID", buildID),
|
||||
zap.Int64("nodeID", nodeID))
|
||||
return indexTaskRetry
|
||||
}
|
||||
// !exist --> node down
|
||||
return indexTaskInProgress
|
||||
return indexTaskRetry
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
|
||||
|
@ -445,32 +492,9 @@ func (ib *indexBuilder) releaseLockAndResetTask(buildID UniqueID, nodeID UniqueI
|
|||
return nil
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) updateStateByMeta(meta *model.SegmentIndex) {
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
|
||||
state, ok := ib.tasks[meta.BuildID]
|
||||
if !ok {
|
||||
log.Warn("index task has been processed", zap.Int64("buildId", meta.BuildID))
|
||||
// no need to return error, this task must have been deleted.
|
||||
return
|
||||
}
|
||||
|
||||
if meta.IndexState == commonpb.IndexState_Finished || meta.IndexState == commonpb.IndexState_Failed {
|
||||
ib.tasks[meta.BuildID] = indexTaskDone
|
||||
log.Info("this task has been finished", zap.Int64("buildID", meta.BuildID),
|
||||
zap.String("original state", state.String()), zap.String("finish or failed", meta.IndexState.String()))
|
||||
return
|
||||
}
|
||||
|
||||
// index state must be Unissued and NodeID is not zero
|
||||
ib.tasks[meta.BuildID] = indexTaskRetry
|
||||
log.Info("this task need to retry", zap.Int64("buildID", meta.BuildID),
|
||||
zap.String("original state", state.String()), zap.String("index state", meta.IndexState.String()),
|
||||
zap.Int64("original nodeID", meta.NodeID))
|
||||
}
|
||||
|
||||
func (ib *indexBuilder) markTasksAsDeleted(buildIDs []UniqueID) {
|
||||
defer ib.notify()
|
||||
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
|
||||
|
@ -483,9 +507,12 @@ func (ib *indexBuilder) markTasksAsDeleted(buildIDs []UniqueID) {
|
|||
}
|
||||
|
||||
func (ib *indexBuilder) nodeDown(nodeID UniqueID) {
|
||||
defer ib.notify()
|
||||
|
||||
metas := ib.meta.GetMetasByNodeID(nodeID)
|
||||
|
||||
ib.taskMutex.Lock()
|
||||
defer ib.taskMutex.Unlock()
|
||||
metas := ib.meta.GetMetasByNodeID(nodeID)
|
||||
|
||||
for _, meta := range metas {
|
||||
if ib.tasks[meta.BuildID] != indexTaskDone {
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -22,12 +22,14 @@ import (
|
|||
"fmt"
|
||||
"math/rand"
|
||||
"os"
|
||||
"strconv"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
@ -78,7 +80,7 @@ type IndexCoord struct {
|
|||
|
||||
factory dependency.Factory
|
||||
etcdCli *clientv3.Client
|
||||
etcdKV *etcdkv.EtcdKV
|
||||
etcdKV kv.MetaKv
|
||||
chunkManager storage.ChunkManager
|
||||
|
||||
metaTable *metaTable
|
||||
|
@ -957,11 +959,11 @@ func (i *IndexCoord) assignTask(builderClient types.IndexNode, req *indexpb.Crea
|
|||
func (i *IndexCoord) createIndexForSegment(segIdx *model.SegmentIndex) (bool, UniqueID, error) {
|
||||
log.Info("create index for flushed segment", zap.Int64("collID", segIdx.CollectionID),
|
||||
zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows))
|
||||
if segIdx.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex {
|
||||
log.Debug("no need to build index", zap.Int64("collID", segIdx.CollectionID),
|
||||
zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows))
|
||||
return false, 0, nil
|
||||
}
|
||||
//if segIdx.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex {
|
||||
// log.Debug("no need to build index", zap.Int64("collID", segIdx.CollectionID),
|
||||
// zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows))
|
||||
// return false, 0, nil
|
||||
//}
|
||||
|
||||
hasIndex, indexBuildID := i.metaTable.HasSameIndex(segIdx.SegmentID, segIdx.IndexID)
|
||||
if hasIndex {
|
||||
|
@ -1038,16 +1040,14 @@ func (i *IndexCoord) watchFlushedSegmentLoop() {
|
|||
for _, event := range events {
|
||||
switch event.Type {
|
||||
case mvccpb.PUT:
|
||||
segmentInfo := &datapb.SegmentInfo{}
|
||||
if err := proto.Unmarshal(event.Kv.Value, segmentInfo); err != nil {
|
||||
log.Error("watchFlushedSegmentLoop unmarshal fail", zap.Error(err))
|
||||
segmentID, err := strconv.ParseInt(string(event.Kv.Value), 10, 64)
|
||||
if err != nil {
|
||||
log.Error("IndexCoord watch flushed segment, but parse segmentID fail",
|
||||
zap.String("event.Value", string(event.Kv.Value)), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
log.Debug("watchFlushedSegmentLoop watch event", zap.Int64("segID", segmentInfo.ID),
|
||||
zap.Int64("collID", segmentInfo.CollectionID), zap.Int64("num rows", segmentInfo.NumOfRows),
|
||||
zap.Int64s("compactForm", segmentInfo.CompactionFrom))
|
||||
i.flushedSegmentWatcher.enqueueInternalTask(segmentInfo)
|
||||
log.Debug("watchFlushedSegmentLoop watch event", zap.Int64("segID", segmentID))
|
||||
i.flushedSegmentWatcher.enqueueInternalTask(segmentID)
|
||||
case mvccpb.DELETE:
|
||||
log.Debug("the segment info has been deleted", zap.String("key", string(event.Kv.Key)))
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@ package indexcoord
|
|||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
|
@ -408,10 +407,43 @@ func NewDataCoordMock() *DataCoordMock {
|
|||
}, nil
|
||||
},
|
||||
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
|
||||
segInfos := make([]*datapb.SegmentInfo, 0)
|
||||
for _, segID := range req.SegmentIDs {
|
||||
segInfos = append(segInfos, &datapb.SegmentInfo{
|
||||
ID: segID,
|
||||
CollectionID: 100,
|
||||
PartitionID: 200,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 1026,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
MaxRowNum: 0,
|
||||
LastExpireTime: 0,
|
||||
StartPosition: nil,
|
||||
DmlPosition: nil,
|
||||
Binlogs: []*datapb.FieldBinlog{
|
||||
{
|
||||
Binlogs: []*datapb.Binlog{
|
||||
{
|
||||
LogPath: "file1",
|
||||
},
|
||||
{
|
||||
LogPath: "file2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
Statslogs: nil,
|
||||
Deltalogs: nil,
|
||||
CreatedByCompaction: false,
|
||||
CompactionFrom: nil,
|
||||
DroppedAt: 0,
|
||||
})
|
||||
}
|
||||
return &datapb.GetSegmentInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
},
|
||||
Infos: segInfos,
|
||||
}, nil
|
||||
},
|
||||
CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
|
||||
|
@ -434,35 +466,6 @@ func NewDataCoordMock() *DataCoordMock {
|
|||
}
|
||||
}
|
||||
|
||||
// ChunkManagerMock is mock
|
||||
// deprecated
|
||||
type ChunkManagerMock struct {
|
||||
storage.ChunkManager
|
||||
|
||||
Fail bool
|
||||
Err bool
|
||||
}
|
||||
|
||||
func (cmm *ChunkManagerMock) Exist(path string) (bool, error) {
|
||||
if cmm.Err {
|
||||
return false, errors.New("path not exist")
|
||||
}
|
||||
if cmm.Fail {
|
||||
return false, nil
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (cmm *ChunkManagerMock) RemoveWithPrefix(prefix string) error {
|
||||
if cmm.Err {
|
||||
return errors.New("error occurred")
|
||||
}
|
||||
if cmm.Fail {
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type mockETCDKV struct {
|
||||
kv.MetaKv
|
||||
|
||||
|
@ -475,6 +478,7 @@ type mockETCDKV struct {
|
|||
loadWithPrefix2 func(key string) ([]string, []string, []int64, error)
|
||||
loadWithPrefix func(key string) ([]string, []string, error)
|
||||
loadWithRevision func(key string) ([]string, []string, int64, error)
|
||||
removeWithPrefix func(key string) error
|
||||
}
|
||||
|
||||
func NewMockEtcdKV() *mockETCDKV {
|
||||
|
@ -500,6 +504,9 @@ func NewMockEtcdKV() *mockETCDKV {
|
|||
loadWithRevision: func(key string) ([]string, []string, int64, error) {
|
||||
return []string{}, []string{}, 0, nil
|
||||
},
|
||||
removeWithPrefix: func(key string) error {
|
||||
return nil
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -539,6 +546,10 @@ func (mk *mockETCDKV) LoadWithRevision(key string) ([]string, []string, int64, e
|
|||
return mk.loadWithRevision(key)
|
||||
}
|
||||
|
||||
func (mk *mockETCDKV) RemoveWithPrefix(key string) error {
|
||||
return mk.removeWithPrefix(key)
|
||||
}
|
||||
|
||||
type chunkManagerMock struct {
|
||||
storage.ChunkManager
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
|
@ -203,48 +202,14 @@ func TestIndexCoord(t *testing.T) {
|
|||
})
|
||||
|
||||
t.Run("FlushedSegmentWatcher", func(t *testing.T) {
|
||||
segInfo := &datapb.SegmentInfo{
|
||||
ID: segID + 1,
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
InsertChannel: "",
|
||||
NumOfRows: 2048,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
MaxRowNum: 4096,
|
||||
LastExpireTime: 0,
|
||||
StartPosition: &internalpb.MsgPosition{
|
||||
Timestamp: createTs,
|
||||
},
|
||||
DmlPosition: nil,
|
||||
Binlogs: []*datapb.FieldBinlog{
|
||||
{
|
||||
FieldID: fieldID,
|
||||
Binlogs: []*datapb.Binlog{
|
||||
{
|
||||
LogPath: "file1",
|
||||
},
|
||||
{
|
||||
LogPath: "file2",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
Statslogs: nil,
|
||||
Deltalogs: nil,
|
||||
CreatedByCompaction: true,
|
||||
CompactionFrom: []int64{segID},
|
||||
DroppedAt: 0,
|
||||
}
|
||||
value, err := proto.Marshal(segInfo)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = ic.etcdKV.Save(path.Join(util.FlushedSegmentPrefix, strconv.FormatInt(collID, 10), strconv.FormatInt(partID, 10), strconv.FormatInt(segInfo.ID, 10)), string(value))
|
||||
segmentID := segID + 1
|
||||
err = ic.etcdKV.Save(path.Join(util.FlushedSegmentPrefix, strconv.FormatInt(collID, 10), strconv.FormatInt(partID, 10), strconv.FormatInt(segmentID, 10)), string(strconv.FormatInt(segmentID, 10)))
|
||||
assert.NoError(t, err)
|
||||
|
||||
req := &indexpb.GetSegmentIndexStateRequest{
|
||||
CollectionID: collID,
|
||||
IndexName: indexName,
|
||||
SegmentIDs: []UniqueID{segInfo.ID},
|
||||
SegmentIDs: []UniqueID{segmentID},
|
||||
}
|
||||
resp, err := ic.GetSegmentIndexState(ctx, req)
|
||||
assert.NoError(t, err)
|
||||
|
|
|
@ -199,17 +199,17 @@ func (mt *metaTable) GetMeta(buildID UniqueID) (*model.SegmentIndex, bool) {
|
|||
return nil, false
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetTypeParams(collID, indexID UniqueID) ([]*commonpb.KeyValuePair, error) {
|
||||
func (mt *metaTable) GetTypeParams(collID, indexID UniqueID) []*commonpb.KeyValuePair {
|
||||
mt.indexLock.RLock()
|
||||
defer mt.indexLock.RUnlock()
|
||||
|
||||
fieldIndexes, ok := mt.collectionIndexes[collID]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("there is no index on collection: %d", collID)
|
||||
return nil
|
||||
}
|
||||
index, ok := fieldIndexes[indexID]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("there is no index on collection: %d with indexID: %d", collID, indexID)
|
||||
return nil
|
||||
}
|
||||
typeParams := make([]*commonpb.KeyValuePair, len(index.TypeParams))
|
||||
|
||||
|
@ -217,7 +217,7 @@ func (mt *metaTable) GetTypeParams(collID, indexID UniqueID) ([]*commonpb.KeyVal
|
|||
typeParams[i] = proto.Clone(param).(*commonpb.KeyValuePair)
|
||||
}
|
||||
|
||||
return typeParams, nil
|
||||
return typeParams
|
||||
}
|
||||
|
||||
func (mt *metaTable) GetIndexParams(collID, indexID UniqueID) []*commonpb.KeyValuePair {
|
||||
|
@ -666,7 +666,7 @@ func (mt *metaTable) GetIndexBuildProgress(indexID int64, createTs uint64) int64
|
|||
continue
|
||||
}
|
||||
|
||||
if segIdx.IndexState == commonpb.IndexState_Finished {
|
||||
if segIdx.IndexState == commonpb.IndexState_Finished && segIdx.NumRows >= Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex {
|
||||
indexRows += segIdx.NumRows
|
||||
}
|
||||
}
|
||||
|
|
|
@ -202,16 +202,13 @@ func TestMetaTable_GetMeta(t *testing.T) {
|
|||
|
||||
func TestMetaTable_GetTypeParams(t *testing.T) {
|
||||
mt := constructMetaTable(&indexcoord.Catalog{})
|
||||
typeParams1, err1 := mt.GetTypeParams(collID, indexID)
|
||||
assert.NoError(t, err1)
|
||||
typeParams1 := mt.GetTypeParams(collID, indexID)
|
||||
assert.Equal(t, 1, len(typeParams1))
|
||||
|
||||
typeParams2, err2 := mt.GetTypeParams(invalidID, indexID)
|
||||
assert.Error(t, err2)
|
||||
typeParams2 := mt.GetTypeParams(invalidID, indexID)
|
||||
assert.Nil(t, typeParams2)
|
||||
|
||||
typeParams3, err3 := mt.GetTypeParams(collID, invalidID)
|
||||
assert.Error(t, err3)
|
||||
typeParams3 := mt.GetTypeParams(collID, invalidID)
|
||||
assert.Nil(t, typeParams3)
|
||||
}
|
||||
|
||||
|
|
|
@ -115,6 +115,7 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest
|
|||
state: info.state,
|
||||
indexFiles: info.indexFiles[:],
|
||||
serializedSize: info.serializedSize,
|
||||
failReason: info.failReason,
|
||||
}
|
||||
}
|
||||
})
|
||||
|
|
|
@ -2,6 +2,8 @@ package indexnode
|
|||
|
||||
import (
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
|
@ -32,6 +34,8 @@ func (i *IndexNode) storeTaskState(ClusterID string, buildID UniqueID, state com
|
|||
i.stateLock.Lock()
|
||||
defer i.stateLock.Unlock()
|
||||
if task, ok := i.tasks[key]; ok {
|
||||
log.Debug("IndexNode store task state", zap.String("clusterID", ClusterID), zap.Int64("buildID", buildID),
|
||||
zap.String("state", state.String()), zap.String("fail reason", failReason))
|
||||
task.state = state
|
||||
task.failReason = failReason
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package datacoord
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
|
@ -102,12 +103,8 @@ func (kc *Catalog) AddSegment(ctx context.Context, segment *datapb.SegmentInfo)
|
|||
|
||||
// save handoff req if segment is flushed
|
||||
if segment.State == commonpb.SegmentState_Flushed {
|
||||
flushedSegmentSegBytes, err := proto.Marshal(segment)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to marshal segment: %d, error: %w", segment.GetID(), err)
|
||||
}
|
||||
flushSegKey := buildFlushedSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
|
||||
kvs[flushSegKey] = string(flushedSegmentSegBytes)
|
||||
kvs[flushSegKey] = strconv.FormatInt(segment.GetID(), 10)
|
||||
}
|
||||
|
||||
return kc.Txn.MultiSave(kvs)
|
||||
|
@ -132,12 +129,8 @@ func (kc *Catalog) AlterSegments(ctx context.Context, modSegments []*datapb.Segm
|
|||
|
||||
// save handoff req if segment is flushed
|
||||
if segment.State == commonpb.SegmentState_Flushed {
|
||||
flushedSegmentSegBytes, err := proto.Marshal(segment)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to marshal segment: %d, error: %w", segment.GetID(), err)
|
||||
}
|
||||
flushSegKey := buildFlushedSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
|
||||
kv[flushSegKey] = string(flushedSegmentSegBytes)
|
||||
kv[flushSegKey] = strconv.FormatInt(segment.GetID(), 10)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,9 +0,0 @@
|
|||
package indexcoord
|
||||
|
||||
const (
|
||||
// SegmentIndexPrefix prefix for segment index meta
|
||||
SegmentIndexPrefix = "segment-index"
|
||||
|
||||
// FieldIndexPrefix prefix for index meta
|
||||
FieldIndexPrefix = "field-index"
|
||||
)
|
|
@ -4,6 +4,8 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -20,11 +22,11 @@ type Catalog struct {
|
|||
}
|
||||
|
||||
func buildIndexKey(collectionID, indexID int64) string {
|
||||
return fmt.Sprintf("%s/%d/%d", FieldIndexPrefix, collectionID, indexID)
|
||||
return fmt.Sprintf("%s/%d/%d", util.FieldIndexPrefix, collectionID, indexID)
|
||||
}
|
||||
|
||||
func buildSegmentIndexKey(collectionID, partitionID, segmentID, buildID int64) string {
|
||||
return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentIndexPrefix, collectionID, partitionID, segmentID, buildID)
|
||||
return fmt.Sprintf("%s/%d/%d/%d/%d", util.SegmentIndexPrefix, collectionID, partitionID, segmentID, buildID)
|
||||
}
|
||||
|
||||
func (kc *Catalog) CreateIndex(ctx context.Context, index *model.Index) error {
|
||||
|
@ -43,9 +45,9 @@ func (kc *Catalog) CreateIndex(ctx context.Context, index *model.Index) error {
|
|||
}
|
||||
|
||||
func (kc *Catalog) ListIndexes(ctx context.Context) ([]*model.Index, error) {
|
||||
_, values, err := kc.Txn.LoadWithPrefix(FieldIndexPrefix)
|
||||
_, values, err := kc.Txn.LoadWithPrefix(util.FieldIndexPrefix)
|
||||
if err != nil {
|
||||
log.Error("list index meta fail", zap.String("prefix", FieldIndexPrefix), zap.Error(err))
|
||||
log.Error("list index meta fail", zap.String("prefix", util.FieldIndexPrefix), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -113,9 +115,9 @@ func (kc *Catalog) CreateSegmentIndex(ctx context.Context, segIdx *model.Segment
|
|||
}
|
||||
|
||||
func (kc *Catalog) ListSegmentIndexes(ctx context.Context) ([]*model.SegmentIndex, error) {
|
||||
_, values, err := kc.Txn.LoadWithPrefix(SegmentIndexPrefix)
|
||||
_, values, err := kc.Txn.LoadWithPrefix(util.SegmentIndexPrefix)
|
||||
if err != nil {
|
||||
log.Error("list segment index meta fail", zap.String("prefix", SegmentIndexPrefix), zap.Error(err))
|
||||
log.Error("list segment index meta fail", zap.String("prefix", util.SegmentIndexPrefix), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
|
|
@ -239,7 +239,7 @@ func (loader *segmentLoader) loadFiles(segment *Segment,
|
|||
if segment.getType() == segmentTypeSealed {
|
||||
fieldID2IndexInfo := make(map[int64]*querypb.FieldIndexInfo)
|
||||
for _, indexInfo := range loadInfo.IndexInfos {
|
||||
if indexInfo != nil && indexInfo.EnableIndex {
|
||||
if indexInfo != nil && indexInfo.EnableIndex && len(indexInfo.IndexFilePaths) > 0 {
|
||||
fieldID := indexInfo.FieldID
|
||||
fieldID2IndexInfo[fieldID] = indexInfo
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue