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
cai.zhang 2022-09-09 15:52:35 +08:00 committed by GitHub
parent eabdc2b114
commit e65ba6331e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 1561 additions and 763 deletions

View File

@ -39,3 +39,7 @@ func msgDataCoordIsUnhealthy(coordID UniqueID) string {
func errDataCoordIsUnhealthy(coordID UniqueID) error { func errDataCoordIsUnhealthy(coordID UniqueID) error {
return errors.New(msgDataCoordIsUnhealthy(coordID)) return errors.New(msgDataCoordIsUnhealthy(coordID))
} }
func msgSegmentNotFound(segID UniqueID) string {
return fmt.Sprintf("failed to get segment %d", segID)
}

View File

@ -37,3 +37,8 @@ func TestErrDataCoordIsUnhealthy(t *testing.T) {
log.Info("TestErrDataCoordIsUnhealthy", zap.Error(errDataCoordIsUnhealthy(nodeID))) log.Info("TestErrDataCoordIsUnhealthy", zap.Error(errDataCoordIsUnhealthy(nodeID)))
} }
} }
func TestErrSegmentNotFound(t *testing.T) {
segID := UniqueID(435846569243121068)
log.Info("TestErrSegmentNotFound", zap.String("msg", msgSegmentNotFound(segID)))
}

View File

@ -350,14 +350,14 @@ func (s *Server) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoR
info = s.meta.GetAllSegment(id) info = s.meta.GetAllSegment(id)
if info == nil { if info == nil {
log.Warn("failed to get segment, this may have been cleaned", zap.Int64("segmentID", id)) 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 return resp, nil
} }
infos = append(infos, info.SegmentInfo) infos = append(infos, info.SegmentInfo)
} else { } else {
info = s.meta.GetSegment(id) info = s.meta.GetSegment(id)
if info == nil { if info == nil {
resp.Status.Reason = fmt.Sprintf("failed to get segment %d", id) resp.Status.Reason = msgSegmentNotFound(id)
return resp, nil return resp, nil
} }
infos = append(infos, info.SegmentInfo) infos = append(infos, info.SegmentInfo)

View File

@ -22,8 +22,9 @@ import (
) )
var ( var (
ErrCompareVersion = errors.New("failed to save meta in etcd because version compare failure") ErrCompareVersion = errors.New("failed to save meta in etcd because version compare failure")
ErrNotIndexExist = errors.New("there is no index") 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. // 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 { func errIndexCoordIsUnhealthy(coordID UniqueID) error {
return errors.New(msgIndexCoordIsUnhealthy(coordID)) 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)
}

View File

@ -17,8 +17,11 @@
package indexcoord package indexcoord
import ( import (
"errors"
"testing" "testing"
"github.com/stretchr/testify/assert"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
@ -48,3 +51,11 @@ func TestErrIndexCoordIsUnhealthy(t *testing.T) {
log.Info("TestErrIndexCoordIsUnhealthy", zap.Error(errIndexCoordIsUnhealthy(nodeID))) 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))
}

View File

@ -18,7 +18,10 @@ package indexcoord
import ( import (
"context" "context"
"errors"
"fmt" "fmt"
"sort"
"strconv"
"sync" "sync"
"time" "time"
@ -79,7 +82,7 @@ func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable,
wg: sync.WaitGroup{}, wg: sync.WaitGroup{},
internalTaskMutex: sync.RWMutex{}, internalTaskMutex: sync.RWMutex{},
childrenTaskMutex: sync.RWMutex{}, childrenTaskMutex: sync.RWMutex{},
scheduleDuration: time.Second * 10, scheduleDuration: time.Second,
internalNotify: make(chan struct{}, 1), internalNotify: make(chan struct{}, 1),
childrenNotify: make(chan struct{}, 1), childrenNotify: make(chan struct{}, 1),
meta: meta, meta: meta,
@ -103,12 +106,12 @@ func (fsw *flushedSegmentWatcher) reloadFromKV() error {
return err return err
} }
for _, value := range values { for _, value := range values {
segmentInfo := &datapb.SegmentInfo{} segID, err := strconv.ParseInt(value, 10, 64)
if err = proto.Unmarshal([]byte(value), segmentInfo); err != nil { if err != nil {
log.Error("flushSegmentWatcher unmarshal segment info fail", zap.Error(err)) log.Error("flushSegmentWatcher parse segmentID fail", zap.String("value", value), zap.Error(err))
return err return err
} }
fsw.enqueueInternalTask(segmentInfo) fsw.enqueueInternalTask(segID)
} }
fsw.etcdRevision = version fsw.etcdRevision = version
return nil return nil
@ -124,23 +127,20 @@ func (fsw *flushedSegmentWatcher) Start() {
func (fsw *flushedSegmentWatcher) Stop() { func (fsw *flushedSegmentWatcher) Stop() {
fsw.cancel() fsw.cancel()
close(fsw.internalNotify)
close(fsw.childrenNotify)
fsw.wg.Wait() fsw.wg.Wait()
} }
func (fsw *flushedSegmentWatcher) enqueueInternalTask(segmentInfo *datapb.SegmentInfo) { func (fsw *flushedSegmentWatcher) enqueueInternalTask(segmentID UniqueID) {
fsw.internalTaskMutex.Lock() fsw.internalTaskMutex.Lock()
defer fsw.internalTaskMutex.Unlock()
fsw.internalTasks[segmentInfo.ID] = &internalTask{ fsw.internalTasks[segmentID] = &internalTask{
segmentInfo: segmentInfo,
state: indexTaskInit, state: indexTaskInit,
segmentInfo: nil,
} }
select { fsw.internalTaskMutex.Unlock()
case fsw.internalNotify <- struct{}{}:
default: fsw.prepare(segmentID)
} fsw.internalNotifyFunc()
} }
func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.SegmentInfo, index *model.Index) { func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.SegmentInfo, index *model.Index) {
@ -172,7 +172,6 @@ func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.Segmen
switch state.state { switch state.state {
case commonpb.IndexState_IndexStateNone: case commonpb.IndexState_IndexStateNone:
fsw.childrenTasks[segmentInfo.ID][index.IndexID].state = indexTaskInit fsw.childrenTasks[segmentInfo.ID][index.IndexID].state = indexTaskInit
case commonpb.IndexState_InProgress, commonpb.IndexState_Unissued, commonpb.IndexState_Retry: case commonpb.IndexState_InProgress, commonpb.IndexState_Unissued, commonpb.IndexState_Retry:
fsw.childrenTasks[segmentInfo.ID][index.IndexID].state = indexTaskInProgress fsw.childrenTasks[segmentInfo.ID][index.IndexID].state = indexTaskInProgress
@ -182,10 +181,7 @@ func (fsw *flushedSegmentWatcher) enqueueChildrenTask(segmentInfo *datapb.Segmen
// can not to here // can not to here
} }
} }
select { fsw.childrenNotifyFunc()
case fsw.childrenNotify <- struct{}{}:
default:
}
} }
func (fsw *flushedSegmentWatcher) internalScheduler() { func (fsw *flushedSegmentWatcher) internalScheduler() {
@ -229,76 +225,164 @@ func (fsw *flushedSegmentWatcher) childrenScheduler() {
} }
func (fsw *flushedSegmentWatcher) internalRun() { func (fsw *flushedSegmentWatcher) internalRun() {
fsw.internalTaskMutex.Lock() fsw.internalTaskMutex.RLock()
defer fsw.internalTaskMutex.Unlock() segmentIDs := make([]UniqueID, 0, len(fsw.internalTasks))
if len(fsw.internalTasks) > 0 { if len(fsw.internalTasks) > 0 {
log.Debug("IndexCoord flushedSegmentWatcher schedule internal tasks", zap.Int("internal task num", len(fsw.internalTasks))) log.Debug("IndexCoord flushedSegmentWatcher schedule internal tasks", zap.Int("internal task num", len(fsw.internalTasks)))
for _, t := range fsw.internalTasks { for segID := range fsw.internalTasks {
fsw.internalProcess(t) 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() { func (fsw *flushedSegmentWatcher) childrenRun() {
fsw.childrenTaskMutex.Lock() fsw.childrenTaskMutex.RLock()
defer fsw.childrenTaskMutex.Unlock() segmentIDs := make([]UniqueID, 0, len(fsw.childrenTasks))
if len(fsw.childrenTasks) > 0 { if len(fsw.childrenTasks) > 0 {
log.Debug("IndexCoord flushedSegmentWatcher schedule children tasks", zap.Int("children task num", len(fsw.childrenTasks))) log.Debug("IndexCoord flushedSegmentWatcher schedule children tasks", zap.Int("children task num", len(fsw.childrenTasks)))
for segID, tasks := range fsw.childrenTasks { for segID := range fsw.childrenTasks {
for _, t := range tasks { segmentIDs = append(segmentIDs, segID)
fsw.childrenProcess(t) }
} sort.Slice(segmentIDs, func(i, j int) bool {
if len(fsw.childrenTasks[segID]) == 0 { return segmentIDs[i] < segmentIDs[j]
delete(fsw.childrenTasks, segID) })
} }
fsw.childrenTaskMutex.RUnlock()
for _, segID := range segmentIDs {
tasks := fsw.getChildrenTasks(segID)
for _, t := range tasks {
fsw.childrenProcess(t)
} }
} }
} }
func (fsw *flushedSegmentWatcher) removeCompactedTasks(t *internalTask) { func (fsw *flushedSegmentWatcher) internalNotifyFunc() {
log.Debug("IndexCoord flushedSegmentWatcher mark task as deleted which is compacted", zap.Int64("segID", t.segmentInfo.ID), select {
zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom)) case fsw.internalNotify <- struct{}{}:
fsw.builder.markTasksAsDeleted(fsw.meta.GetBuildIDsFromSegIDs(t.segmentInfo.CompactionFrom)) default:
for _, segID := range t.segmentInfo.CompactionFrom { }
if _, ok := fsw.internalTasks[segID]; ok { }
fsw.internalTasks[segID].state = indexTaskDeleted
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) { func (fsw *flushedSegmentWatcher) hasChildrenTaskDone(segID UniqueID) bool {
log.Debug("IndexCoord flushedSegmentWatcher process internal task", zap.Int64("segID", t.segmentInfo.ID), 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())) zap.String("state", t.state.String()))
switch t.state { switch t.state {
case indexTaskInit: case indexTaskInit:
if t.segmentInfo.CreatedByCompaction { fsw.constructTask(t)
fsw.removeCompactedTasks(t) fsw.updateInternalTaskState(segID, indexTaskInProgress)
} fsw.internalNotifyFunc()
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
case indexTaskInProgress: case indexTaskInProgress:
fsw.childrenTaskMutex.RLock() if fsw.hasChildrenTaskDone(segID) {
defer fsw.childrenTaskMutex.RUnlock() fsw.updateInternalTaskState(segID, indexTaskDone)
if tasks, ok := fsw.childrenTasks[t.segmentInfo.ID]; !ok || len(tasks) == 0 { fsw.internalNotifyFunc()
fsw.internalTasks[t.segmentInfo.ID].state = indexTaskDone
select {
case fsw.internalNotify <- struct{}{}:
default:
}
} }
return
case indexTaskDone: case indexTaskDone:
handoffTask := &querypb.SegmentInfo{ handoffTask := &querypb.SegmentInfo{
SegmentID: t.segmentInfo.ID, SegmentID: segID,
CollectionID: t.segmentInfo.CollectionID, CollectionID: t.segmentInfo.CollectionID,
PartitionID: t.segmentInfo.PartitionID, PartitionID: t.segmentInfo.PartitionID,
NumRows: t.segmentInfo.NumOfRows, NumRows: t.segmentInfo.NumOfRows,
@ -313,25 +397,21 @@ func (fsw *flushedSegmentWatcher) internalProcess(t *internalTask) {
} }
if err := fsw.writeHandoffSegment(handoffTask); err != nil { if err := fsw.writeHandoffSegment(handoffTask); err != nil {
log.Error("IndexCoord flushSegmentWatcher writeHandoffSegment with no index fail", 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 return
} }
if err := fsw.removeFlushedSegment(t); err != nil { if err := fsw.removeFlushedSegment(t); err != nil {
log.Error("IndexCoord flushSegmentWatcher removeFlushedSegment fail", log.Error("IndexCoord flushSegmentWatcher removeFlushedSegment fail",
zap.Int64("segID", t.segmentInfo.ID), zap.Error(err)) zap.Int64("segID", segID), zap.Error(err))
return return
} }
delete(fsw.internalTasks, t.segmentInfo.ID) fsw.deleteInternalTask(segID)
return fsw.internalNotifyFunc()
case indexTaskDeleted: case indexTaskDeleted:
if t.segmentInfo.CreatedByCompaction { if t.segmentInfo.CreatedByCompaction {
fsw.removeCompactedTasks(t) fsw.removeCompactedTasks(t)
} }
fsw.childrenTaskMutex.Lock() fsw.updateInternalTaskState(segID, indexTaskDone)
delete(fsw.childrenTasks, t.segmentInfo.ID)
fsw.childrenTaskMutex.Unlock()
fsw.internalTasks[t.segmentInfo.ID].state = indexTaskDone
return
default: default:
log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", t.segmentInfo.ID), log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", t.segmentInfo.ID),
zap.String("state", t.state.String())) zap.String("state", t.state.String()))
@ -375,26 +455,34 @@ func (fsw *flushedSegmentWatcher) childrenProcess(task *childrenTask) {
fsw.builder.enqueue(buildID) fsw.builder.enqueue(buildID)
} }
fsw.childrenTasks[segID][task.indexInfo.IndexID].state = indexTaskInProgress fsw.updateChildrenTaskState(segID, task.indexInfo.IndexID, indexTaskInProgress)
return fsw.childrenNotifyFunc()
case indexTaskInProgress: case indexTaskInProgress:
filePath, err := fsw.meta.GetIndexFilePathInfo(segID, task.indexInfo.IndexID) state := fsw.meta.GetSegmentIndexState(task.segmentInfo.ID, task.indexInfo.IndexID)
if err != nil { if state.state == commonpb.IndexState_IndexStateNone {
log.Warn("IndexCoord get index file path fail", zap.Int64("collID", task.segmentInfo.CollectionID), log.Debug("task is no need to build index, remove task", zap.Int64("segID", task.segmentInfo.ID),
zap.Int64("partID", task.segmentInfo.PartitionID), zap.Int64("segID", segID), zap.Error(err)) zap.Int64("indexID", task.indexInfo.IndexID))
fsw.deleteChildTask(task.segmentInfo.ID, task.indexInfo.IndexID)
fsw.childrenNotifyFunc()
return return
} }
fsw.childrenTasks[segID][task.indexInfo.IndexID].indexInfo.IndexFilePaths = filePath.IndexFilePaths if state.state != commonpb.IndexState_Finished && state.state != commonpb.IndexState_Failed {
fsw.childrenTasks[segID][task.indexInfo.IndexID].indexInfo.IndexSize = int64(filePath.SerializedSize) log.Debug("the index on segment is not finish", zap.Int64("segID", segID),
fsw.childrenTasks[segID][task.indexInfo.IndexID].state = indexTaskDone zap.String("state", state.state.String()), zap.String("fail reason", state.failReason))
return
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: case indexTaskDone:
handoffTask := &querypb.SegmentInfo{ handoffTask := &querypb.SegmentInfo{
SegmentID: task.segmentInfo.ID, SegmentID: task.segmentInfo.ID,
CollectionID: task.segmentInfo.CollectionID, CollectionID: task.segmentInfo.CollectionID,
PartitionID: task.segmentInfo.PartitionID, PartitionID: task.segmentInfo.PartitionID,
NumRows: task.segmentInfo.NumOfRows, NumRows: task.segmentInfo.NumOfRows,
DmChannel: task.segmentInfo.GetInsertChannel(),
IndexName: task.indexInfo.IndexName, IndexName: task.indexInfo.IndexName,
IndexID: task.indexInfo.IndexID, IndexID: task.indexInfo.IndexID,
CompactionFrom: task.segmentInfo.CompactionFrom, 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)) zap.Int64("partID", task.segmentInfo.PartitionID), zap.Int64("segID", task.segmentInfo.ID), zap.Error(err))
return 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)) zap.Int64("partID", task.segmentInfo.PartitionID), zap.Int64("segID", task.segmentInfo.ID))
delete(fsw.childrenTasks[task.segmentInfo.ID], task.indexInfo.IndexID) fsw.deleteChildTask(task.segmentInfo.ID, task.indexInfo.IndexID)
return fsw.childrenNotifyFunc()
default: default:
log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", task.segmentInfo.ID), log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", task.segmentInfo.ID),
zap.String("state", task.state.String())) 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), log.Debug("IndexCoord flushedSegmentWatcher construct tasks by segment info", zap.Int64("segID", t.segmentInfo.ID),
zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom)) zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
fieldIndexes := fsw.meta.GetIndexesForCollection(t.segmentInfo.CollectionID, "") 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), 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))) zap.Int64("num of rows", t.segmentInfo.NumOfRows), zap.Int("collection indexes num", len(fieldIndexes)))
// no need to build index // no need to build index
return nil return
} }
for _, index := range fieldIndexes { for _, index := range fieldIndexes {
fsw.enqueueChildrenTask(t.segmentInfo, index) fsw.enqueueChildrenTask(t.segmentInfo, index)
} }
return nil
} }
func (fsw *flushedSegmentWatcher) writeHandoffSegment(t *querypb.SegmentInfo) error { 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 { 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) deletedKeys := fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, t.segmentInfo.CollectionID, t.segmentInfo.PartitionID, t.segmentInfo.ID)
err := fsw.kvClient.RemoveWithPrefix(deletedKeys) err := fsw.kvClient.RemoveWithPrefix(deletedKeys)
if err != nil { 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)) zap.Int64("partID", t.segmentInfo.PartitionID), zap.Int64("segID", t.segmentInfo.ID))
return nil 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)
}
}
}

View File

@ -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 package indexcoord
import ( import (
"context" "context"
"errors"
"sync"
"testing" "testing"
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
) )
func Test_flushSegmentWatcher(t *testing.T) { func Test_flushSegmentWatcher(t *testing.T) {
ctx := context.Background() 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{ watcher, err := newFlushSegmentWatcher(ctx,
loadWithRevision: func(key string) ([]string, []string, int64, error) { &mockETCDKV{
return []string{"seg1"}, []string{string(value)}, 1, nil loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{"seg1"}, []string{"12345"}, 1, nil
},
}, },
},
&metaTable{ &metaTable{
catalog: &indexcoord.Catalog{ catalog: &indexcoord.Catalog{
Txn: NewMockEtcdKV(), Txn: NewMockEtcdKV(),
}, },
}, },
&indexBuilder{}, &IndexCoord{}) &indexBuilder{}, &IndexCoord{
dataCoordClient: NewDataCoordMock(),
})
assert.NoError(t, err) assert.NoError(t, err)
assert.NotNil(t, watcher) 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)
})
}

View File

@ -19,17 +19,16 @@ package indexcoord
import ( import (
"context" "context"
"path" "path"
"sort"
"sync" "sync"
"time" "time"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/proto/datapb"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/indexpb"
) )
@ -42,8 +41,8 @@ type indexBuilder struct {
scheduleDuration time.Duration scheduleDuration time.Duration
// TODO @xiaocai2333: use priority queue // TODO @xiaocai2333: use priority queue
tasks map[int64]indexTaskState tasks map[int64]indexTaskState
notify chan struct{} notifyChan chan struct{}
ic *IndexCoord ic *IndexCoord
@ -58,9 +57,9 @@ func newIndexBuilder(ctx context.Context, ic *IndexCoord, metaTable *metaTable,
cancel: cancel, cancel: cancel,
meta: metaTable, meta: metaTable,
ic: ic, ic: ic,
tasks: make(map[int64]indexTaskState, 1024), tasks: make(map[int64]indexTaskState),
notify: make(chan struct{}, 1), notifyChan: make(chan struct{}, 1),
scheduleDuration: time.Second, scheduleDuration: time.Second * 3,
} }
ib.reloadFromKV(aliveNodes) ib.reloadFromKV(aliveNodes)
return ib return ib
@ -73,7 +72,6 @@ func (ib *indexBuilder) Start() {
func (ib *indexBuilder) Stop() { func (ib *indexBuilder) Stop() {
ib.cancel() ib.cancel()
close(ib.notify)
ib.wg.Wait() ib.wg.Wait()
} }
@ -118,19 +116,24 @@ func (ib *indexBuilder) reloadFromKV(aliveNodes []UniqueID) {
} }
} }
func (ib *indexBuilder) enqueue(buildID UniqueID) { // notify is an unblocked notify function
// notify func (ib *indexBuilder) notify() {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
ib.tasks[buildID] = indexTaskInit
select { select {
case ib.notify <- struct{}{}: case ib.notifyChan <- struct{}{}:
default: 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() { func (ib *indexBuilder) schedule() {
// receive notify // receive notifyChan
// time ticker // time ticker
log.Info("index builder schedule loop start") log.Info("index builder schedule loop start")
defer ib.wg.Done() defer ib.wg.Done()
@ -141,43 +144,82 @@ func (ib *indexBuilder) schedule() {
case <-ib.ctx.Done(): case <-ib.ctx.Done():
log.Warn("index builder ctx done") log.Warn("index builder ctx done")
return return
case _, ok := <-ib.notify: case _, ok := <-ib.notifyChan:
if ok { if ok {
ib.taskMutex.Lock() ib.run()
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()
} }
// !ok means indexBuilder is closed. // !ok means indexBuild is closed.
case <-ticker.C: case <-ticker.C:
ib.taskMutex.Lock() ib.run()
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()
} }
} }
} }
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) { func (ib *indexBuilder) process(buildID UniqueID) {
defer ib.notify()
ib.taskMutex.RLock()
state := ib.tasks[buildID] 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())) log.Info("index task is processing", zap.Int64("buildID", buildID), zap.String("task state", state.String()))
meta, exist := ib.meta.GetMeta(buildID) 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 { switch state {
case indexTaskInit: case indexTaskInit:
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) { if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
delete(ib.tasks, buildID) 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 return
} }
log.Debug("task state is init, build index ...", zap.Int64("buildID", buildID))
// peek client // peek client
// if all IndexNodes are executing task, wait for one of them to finish the task. // if all IndexNodes are executing task, wait for one of them to finish the task.
nodeID, client := ib.ic.nodeManager.PeekClient(meta) 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 { 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), log.Error("index builder acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err)) zap.Int64("nodeID", nodeID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry updateStateFunc(buildID, indexTaskRetry)
return return
} }
segmentsInfo, err := ib.ic.dataCoordClient.GetSegmentInfo(ib.ctx, &datapb.GetSegmentInfoRequest{ segmentsInfo, err := ib.ic.dataCoordClient.GetSegmentInfo(ib.ctx, &datapb.GetSegmentInfoRequest{
@ -206,13 +248,17 @@ func (ib *indexBuilder) process(buildID UniqueID) {
if err != nil { if err != nil {
log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID), log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID),
zap.Int64("buildID", buildID), zap.Error(err)) zap.Int64("buildID", buildID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry updateStateFunc(buildID, indexTaskRetry)
return return
} }
if segmentsInfo.Status.ErrorCode != commonpb.ErrorCode_Success { if segmentsInfo.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID), log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID),
zap.Int64("buildID", buildID), zap.String("failReason", segmentsInfo.Status.Reason)) 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 return
} }
binLogs := make([]string, 0) binLogs := make([]string, 0)
@ -232,12 +278,7 @@ func (ib *indexBuilder) process(buildID UniqueID) {
break break
} }
typeParams, err := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID) typeParams := 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
}
indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID) indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID)
var storageConfig *indexpb.StorageConfig 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 // 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), log.Error("index builder assign task to IndexNode failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err)) zap.Int64("nodeID", nodeID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry updateStateFunc(buildID, indexTaskRetry)
return return
} }
// update index meta state to InProgress // 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 // 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), log.Error("index builder update index meta to InProgress failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err)) zap.Int64("nodeID", nodeID), zap.Error(err))
ib.tasks[buildID] = indexTaskRetry updateStateFunc(buildID, indexTaskRetry)
return return
} }
log.Debug("index task assigned success", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID)) log.Debug("index task assigned success", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
ib.tasks[buildID] = indexTaskInProgress updateStateFunc(buildID, indexTaskInProgress)
case indexTaskDone: case indexTaskDone:
log.Debug("index task has done", zap.Int64("buildID", buildID)) log.Debug("index task has done", zap.Int64("buildID", buildID))
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) { if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
ib.tasks[buildID] = indexTaskDeleted updateStateFunc(buildID, indexTaskDeleted)
return return
} }
@ -304,11 +345,11 @@ func (ib *indexBuilder) process(buildID UniqueID) {
log.Error("index builder try to release reference lock failed", zap.Error(err)) log.Error("index builder try to release reference lock failed", zap.Error(err))
return return
} }
delete(ib.tasks, buildID) deleteFunc(buildID)
case indexTaskRetry: case indexTaskRetry:
log.Debug("index task state is retry, try to release reference lock", zap.Int64("buildID", buildID)) 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) { if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
ib.tasks[buildID] = indexTaskDeleted updateStateFunc(buildID, indexTaskDeleted)
return return
} }
if err := ib.releaseLockAndResetTask(buildID, meta.NodeID); err != nil { 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)) log.Error("index builder try to release reference lock failed", zap.Error(err))
return return
} }
ib.tasks[buildID] = indexTaskInit
updateStateFunc(buildID, indexTaskInit)
case indexTaskDeleted: case indexTaskDeleted:
log.Debug("index task state is deleted, try to release reference lock", zap.Int64("buildID", buildID)) 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. // reset nodeID success, remove task.
delete(ib.tasks, buildID) deleteFunc(buildID)
default: default:
log.Debug("index task is in progress", zap.Int64("buildID", buildID), log.Debug("index task is in progress", zap.Int64("buildID", buildID),
zap.String("state", meta.IndexState.String())) zap.String("state", meta.IndexState.String()))
if !exist || !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) { if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
ib.tasks[buildID] = indexTaskDeleted updateStateFunc(buildID, indexTaskDeleted)
return 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. // indexInfos length is always one.
for _, info := range response.IndexInfos { for _, info := range response.IndexInfos {
if info.State == commonpb.IndexState_Failed || info.State == commonpb.IndexState_Finished { if info.BuildID == buildID {
log.Info("this task has been finished", zap.Int64("buildID", info.BuildID), if info.State == commonpb.IndexState_Failed || info.State == commonpb.IndexState_Finished {
zap.String("index state", info.State.String())) log.Info("this task has been finished", zap.Int64("buildID", info.BuildID),
if err := ib.meta.FinishTask(info); err != nil { zap.String("index state", info.State.String()))
log.Error("IndexCoord update index state fail", zap.Int64("buildID", info.BuildID), if err := ib.meta.FinishTask(info); err != nil {
zap.String("index state", info.State.String()), zap.Error(err)) log.Error("IndexCoord update index state fail", zap.Int64("buildID", info.BuildID),
return indexTaskInProgress 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 return indexTaskInProgress
} else if info.State == commonpb.IndexState_Retry {
log.Info("this task should be retry", zap.Int64("buildID", buildID))
return indexTaskRetry
} }
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 // !exist --> node down
return indexTaskInProgress return indexTaskRetry
} }
func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool { func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
@ -445,32 +492,9 @@ func (ib *indexBuilder) releaseLockAndResetTask(buildID UniqueID, nodeID UniqueI
return nil 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) { func (ib *indexBuilder) markTasksAsDeleted(buildIDs []UniqueID) {
defer ib.notify()
ib.taskMutex.Lock() ib.taskMutex.Lock()
defer ib.taskMutex.Unlock() defer ib.taskMutex.Unlock()
@ -483,9 +507,12 @@ func (ib *indexBuilder) markTasksAsDeleted(buildIDs []UniqueID) {
} }
func (ib *indexBuilder) nodeDown(nodeID UniqueID) { func (ib *indexBuilder) nodeDown(nodeID UniqueID) {
defer ib.notify()
metas := ib.meta.GetMetasByNodeID(nodeID)
ib.taskMutex.Lock() ib.taskMutex.Lock()
defer ib.taskMutex.Unlock() defer ib.taskMutex.Unlock()
metas := ib.meta.GetMetasByNodeID(nodeID)
for _, meta := range metas { for _, meta := range metas {
if ib.tasks[meta.BuildID] != indexTaskDone { if ib.tasks[meta.BuildID] != indexTaskDone {

File diff suppressed because it is too large Load Diff

View File

@ -22,12 +22,14 @@ import (
"fmt" "fmt"
"math/rand" "math/rand"
"os" "os"
"strconv"
"sync" "sync"
"sync/atomic" "sync/atomic"
"syscall" "syscall"
"time" "time"
"github.com/golang/protobuf/proto" "github.com/milvus-io/milvus/internal/kv"
"go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/mvccpb"
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
clientv3 "go.etcd.io/etcd/client/v3" clientv3 "go.etcd.io/etcd/client/v3"
@ -78,7 +80,7 @@ type IndexCoord struct {
factory dependency.Factory factory dependency.Factory
etcdCli *clientv3.Client etcdCli *clientv3.Client
etcdKV *etcdkv.EtcdKV etcdKV kv.MetaKv
chunkManager storage.ChunkManager chunkManager storage.ChunkManager
metaTable *metaTable 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) { func (i *IndexCoord) createIndexForSegment(segIdx *model.SegmentIndex) (bool, UniqueID, error) {
log.Info("create index for flushed segment", zap.Int64("collID", segIdx.CollectionID), log.Info("create index for flushed segment", zap.Int64("collID", segIdx.CollectionID),
zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows)) zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows))
if segIdx.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex { //if segIdx.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex {
log.Debug("no need to build index", zap.Int64("collID", segIdx.CollectionID), // log.Debug("no need to build index", zap.Int64("collID", segIdx.CollectionID),
zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows)) // zap.Int64("segID", segIdx.SegmentID), zap.Int64("numRows", segIdx.NumRows))
return false, 0, nil // return false, 0, nil
} //}
hasIndex, indexBuildID := i.metaTable.HasSameIndex(segIdx.SegmentID, segIdx.IndexID) hasIndex, indexBuildID := i.metaTable.HasSameIndex(segIdx.SegmentID, segIdx.IndexID)
if hasIndex { if hasIndex {
@ -1038,16 +1040,14 @@ func (i *IndexCoord) watchFlushedSegmentLoop() {
for _, event := range events { for _, event := range events {
switch event.Type { switch event.Type {
case mvccpb.PUT: case mvccpb.PUT:
segmentInfo := &datapb.SegmentInfo{} segmentID, err := strconv.ParseInt(string(event.Kv.Value), 10, 64)
if err := proto.Unmarshal(event.Kv.Value, segmentInfo); err != nil { if err != nil {
log.Error("watchFlushedSegmentLoop unmarshal fail", zap.Error(err)) log.Error("IndexCoord watch flushed segment, but parse segmentID fail",
zap.String("event.Value", string(event.Kv.Value)), zap.Error(err))
continue continue
} }
log.Debug("watchFlushedSegmentLoop watch event", zap.Int64("segID", segmentID))
log.Debug("watchFlushedSegmentLoop watch event", zap.Int64("segID", segmentInfo.ID), i.flushedSegmentWatcher.enqueueInternalTask(segmentID)
zap.Int64("collID", segmentInfo.CollectionID), zap.Int64("num rows", segmentInfo.NumOfRows),
zap.Int64s("compactForm", segmentInfo.CompactionFrom))
i.flushedSegmentWatcher.enqueueInternalTask(segmentInfo)
case mvccpb.DELETE: case mvccpb.DELETE:
log.Debug("the segment info has been deleted", zap.String("key", string(event.Kv.Key))) log.Debug("the segment info has been deleted", zap.String("key", string(event.Kv.Key)))
} }

View File

@ -18,7 +18,6 @@ package indexcoord
import ( import (
"context" "context"
"errors"
"math/rand" "math/rand"
"time" "time"
@ -408,10 +407,43 @@ func NewDataCoordMock() *DataCoordMock {
}, nil }, nil
}, },
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { 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{ return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success, ErrorCode: commonpb.ErrorCode_Success,
}, },
Infos: segInfos,
}, nil }, nil
}, },
CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) { 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 { type mockETCDKV struct {
kv.MetaKv kv.MetaKv
@ -475,6 +478,7 @@ type mockETCDKV struct {
loadWithPrefix2 func(key string) ([]string, []string, []int64, error) loadWithPrefix2 func(key string) ([]string, []string, []int64, error)
loadWithPrefix func(key string) ([]string, []string, error) loadWithPrefix func(key string) ([]string, []string, error)
loadWithRevision func(key string) ([]string, []string, int64, error) loadWithRevision func(key string) ([]string, []string, int64, error)
removeWithPrefix func(key string) error
} }
func NewMockEtcdKV() *mockETCDKV { func NewMockEtcdKV() *mockETCDKV {
@ -500,6 +504,9 @@ func NewMockEtcdKV() *mockETCDKV {
loadWithRevision: func(key string) ([]string, []string, int64, error) { loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{}, []string{}, 0, nil 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) return mk.loadWithRevision(key)
} }
func (mk *mockETCDKV) RemoveWithPrefix(key string) error {
return mk.removeWithPrefix(key)
}
type chunkManagerMock struct { type chunkManagerMock struct {
storage.ChunkManager storage.ChunkManager

View File

@ -24,7 +24,6 @@ import (
"testing" "testing"
"time" "time"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/common"
@ -203,48 +202,14 @@ func TestIndexCoord(t *testing.T) {
}) })
t.Run("FlushedSegmentWatcher", func(t *testing.T) { t.Run("FlushedSegmentWatcher", func(t *testing.T) {
segInfo := &datapb.SegmentInfo{ segmentID := segID + 1
ID: 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)))
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))
assert.NoError(t, err) assert.NoError(t, err)
req := &indexpb.GetSegmentIndexStateRequest{ req := &indexpb.GetSegmentIndexStateRequest{
CollectionID: collID, CollectionID: collID,
IndexName: indexName, IndexName: indexName,
SegmentIDs: []UniqueID{segInfo.ID}, SegmentIDs: []UniqueID{segmentID},
} }
resp, err := ic.GetSegmentIndexState(ctx, req) resp, err := ic.GetSegmentIndexState(ctx, req)
assert.NoError(t, err) assert.NoError(t, err)

View File

@ -199,17 +199,17 @@ func (mt *metaTable) GetMeta(buildID UniqueID) (*model.SegmentIndex, bool) {
return nil, false 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() mt.indexLock.RLock()
defer mt.indexLock.RUnlock() defer mt.indexLock.RUnlock()
fieldIndexes, ok := mt.collectionIndexes[collID] fieldIndexes, ok := mt.collectionIndexes[collID]
if !ok { if !ok {
return nil, fmt.Errorf("there is no index on collection: %d", collID) return nil
} }
index, ok := fieldIndexes[indexID] index, ok := fieldIndexes[indexID]
if !ok { 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)) 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) typeParams[i] = proto.Clone(param).(*commonpb.KeyValuePair)
} }
return typeParams, nil return typeParams
} }
func (mt *metaTable) GetIndexParams(collID, indexID UniqueID) []*commonpb.KeyValuePair { func (mt *metaTable) GetIndexParams(collID, indexID UniqueID) []*commonpb.KeyValuePair {
@ -666,7 +666,7 @@ func (mt *metaTable) GetIndexBuildProgress(indexID int64, createTs uint64) int64
continue continue
} }
if segIdx.IndexState == commonpb.IndexState_Finished { if segIdx.IndexState == commonpb.IndexState_Finished && segIdx.NumRows >= Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex {
indexRows += segIdx.NumRows indexRows += segIdx.NumRows
} }
} }

View File

@ -202,16 +202,13 @@ func TestMetaTable_GetMeta(t *testing.T) {
func TestMetaTable_GetTypeParams(t *testing.T) { func TestMetaTable_GetTypeParams(t *testing.T) {
mt := constructMetaTable(&indexcoord.Catalog{}) mt := constructMetaTable(&indexcoord.Catalog{})
typeParams1, err1 := mt.GetTypeParams(collID, indexID) typeParams1 := mt.GetTypeParams(collID, indexID)
assert.NoError(t, err1)
assert.Equal(t, 1, len(typeParams1)) assert.Equal(t, 1, len(typeParams1))
typeParams2, err2 := mt.GetTypeParams(invalidID, indexID) typeParams2 := mt.GetTypeParams(invalidID, indexID)
assert.Error(t, err2)
assert.Nil(t, typeParams2) assert.Nil(t, typeParams2)
typeParams3, err3 := mt.GetTypeParams(collID, invalidID) typeParams3 := mt.GetTypeParams(collID, invalidID)
assert.Error(t, err3)
assert.Nil(t, typeParams3) assert.Nil(t, typeParams3)
} }

View File

@ -115,6 +115,7 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest
state: info.state, state: info.state,
indexFiles: info.indexFiles[:], indexFiles: info.indexFiles[:],
serializedSize: info.serializedSize, serializedSize: info.serializedSize,
failReason: info.failReason,
} }
} }
}) })

View File

@ -2,6 +2,8 @@ package indexnode
import ( import (
"github.com/golang/protobuf/proto" "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/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb" "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() i.stateLock.Lock()
defer i.stateLock.Unlock() defer i.stateLock.Unlock()
if task, ok := i.tasks[key]; ok { 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.state = state
task.failReason = failReason task.failReason = failReason
} }

View File

@ -19,6 +19,7 @@ package datacoord
import ( import (
"context" "context"
"fmt" "fmt"
"strconv"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv" "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 // save handoff req if segment is flushed
if segment.State == commonpb.SegmentState_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()) flushSegKey := buildFlushedSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
kvs[flushSegKey] = string(flushedSegmentSegBytes) kvs[flushSegKey] = strconv.FormatInt(segment.GetID(), 10)
} }
return kc.Txn.MultiSave(kvs) 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 // save handoff req if segment is flushed
if segment.State == commonpb.SegmentState_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()) flushSegKey := buildFlushedSegmentPath(segment.GetCollectionID(), segment.GetPartitionID(), segment.GetID())
kv[flushSegKey] = string(flushedSegmentSegBytes) kv[flushSegKey] = strconv.FormatInt(segment.GetID(), 10)
} }
} }

View File

@ -1,9 +0,0 @@
package indexcoord
const (
// SegmentIndexPrefix prefix for segment index meta
SegmentIndexPrefix = "segment-index"
// FieldIndexPrefix prefix for index meta
FieldIndexPrefix = "field-index"
)

View File

@ -4,6 +4,8 @@ import (
"context" "context"
"fmt" "fmt"
"github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/log"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
@ -20,11 +22,11 @@ type Catalog struct {
} }
func buildIndexKey(collectionID, indexID int64) string { 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 { 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 { 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) { 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 { 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 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) { 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 { 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 return nil, err
} }

View File

@ -239,7 +239,7 @@ func (loader *segmentLoader) loadFiles(segment *Segment,
if segment.getType() == segmentTypeSealed { if segment.getType() == segmentTypeSealed {
fieldID2IndexInfo := make(map[int64]*querypb.FieldIndexInfo) fieldID2IndexInfo := make(map[int64]*querypb.FieldIndexInfo)
for _, indexInfo := range loadInfo.IndexInfos { for _, indexInfo := range loadInfo.IndexInfos {
if indexInfo != nil && indexInfo.EnableIndex { if indexInfo != nil && indexInfo.EnableIndex && len(indexInfo.IndexFilePaths) > 0 {
fieldID := indexInfo.FieldID fieldID := indexInfo.FieldID
fieldID2IndexInfo[fieldID] = indexInfo fieldID2IndexInfo[fieldID] = indexInfo
} }