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

View File

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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