2022-08-25 07:48:54 +00:00
|
|
|
// 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"
|
2022-09-09 07:52:35 +00:00
|
|
|
"errors"
|
2022-08-25 07:48:54 +00:00
|
|
|
"fmt"
|
2022-09-09 07:52:35 +00:00
|
|
|
"sort"
|
|
|
|
"strconv"
|
2022-08-25 07:48:54 +00:00
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
|
|
|
|
"github.com/golang/protobuf/proto"
|
|
|
|
clientv3 "go.etcd.io/etcd/client/v3"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2022-09-16 08:56:49 +00:00
|
|
|
"github.com/milvus-io/milvus/api/commonpb"
|
2022-08-25 07:48:54 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/kv"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/metastore/model"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
|
|
|
"github.com/milvus-io/milvus/internal/util"
|
2022-09-15 07:44:31 +00:00
|
|
|
"github.com/milvus-io/milvus/internal/util/logutil"
|
2022-08-25 07:48:54 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
type flushedSegmentWatcher struct {
|
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
|
|
|
|
|
|
|
kvClient kv.MetaKv
|
|
|
|
wg sync.WaitGroup
|
|
|
|
scheduleDuration time.Duration
|
|
|
|
|
|
|
|
internalTaskMutex sync.RWMutex
|
|
|
|
internalNotify chan struct{}
|
|
|
|
|
|
|
|
etcdRevision int64
|
|
|
|
watchChan clientv3.WatchChan
|
|
|
|
|
|
|
|
meta *metaTable
|
|
|
|
builder *indexBuilder
|
|
|
|
ic *IndexCoord
|
|
|
|
|
|
|
|
internalTasks map[UniqueID]*internalTask
|
|
|
|
}
|
|
|
|
|
|
|
|
type internalTask struct {
|
|
|
|
state indexTaskState
|
|
|
|
segmentInfo *datapb.SegmentInfo
|
|
|
|
}
|
|
|
|
|
|
|
|
func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable, builder *indexBuilder, ic *IndexCoord) (*flushedSegmentWatcher, error) {
|
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
fsw := &flushedSegmentWatcher{
|
|
|
|
ctx: ctx,
|
|
|
|
cancel: cancel,
|
|
|
|
kvClient: kv,
|
|
|
|
wg: sync.WaitGroup{},
|
|
|
|
internalTaskMutex: sync.RWMutex{},
|
2022-09-09 07:52:35 +00:00
|
|
|
scheduleDuration: time.Second,
|
2022-08-25 07:48:54 +00:00
|
|
|
internalNotify: make(chan struct{}, 1),
|
|
|
|
meta: meta,
|
|
|
|
builder: builder,
|
|
|
|
ic: ic,
|
|
|
|
}
|
|
|
|
err := fsw.reloadFromKV()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return fsw, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) reloadFromKV() error {
|
|
|
|
log.Info("flushSegmentWatcher reloadFromKV")
|
|
|
|
fsw.internalTasks = make(map[UniqueID]*internalTask)
|
|
|
|
_, values, version, err := fsw.kvClient.LoadWithRevision(util.FlushedSegmentPrefix)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("flushSegmentWatcher reloadFromKV fail", zap.String("prefix", util.FlushedSegmentPrefix), zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
for _, value := range values {
|
2022-09-09 07:52:35 +00:00
|
|
|
segID, err := strconv.ParseInt(value, 10, 64)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("flushSegmentWatcher parse segmentID fail", zap.String("value", value), zap.Error(err))
|
2022-08-25 07:48:54 +00:00
|
|
|
return err
|
|
|
|
}
|
2022-09-09 07:52:35 +00:00
|
|
|
fsw.enqueueInternalTask(segID)
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
fsw.etcdRevision = version
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) Start() {
|
|
|
|
fsw.wg.Add(1)
|
|
|
|
go fsw.internalScheduler()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) Stop() {
|
|
|
|
fsw.cancel()
|
|
|
|
fsw.wg.Wait()
|
|
|
|
}
|
|
|
|
|
2022-09-09 07:52:35 +00:00
|
|
|
func (fsw *flushedSegmentWatcher) enqueueInternalTask(segmentID UniqueID) {
|
2022-09-15 07:44:31 +00:00
|
|
|
defer fsw.internalNotifyFunc()
|
2022-08-25 07:48:54 +00:00
|
|
|
fsw.internalTaskMutex.Lock()
|
2022-09-15 07:44:31 +00:00
|
|
|
defer fsw.internalTaskMutex.Unlock()
|
2022-08-25 07:48:54 +00:00
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
logutil.Logger(fsw.ctx).Info("flushedSegmentWatcher enqueueInternalTask", zap.Int64("segmentID", segmentID))
|
2022-08-25 07:48:54 +00:00
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
if _, ok := fsw.internalTasks[segmentID]; !ok {
|
|
|
|
fsw.internalTasks[segmentID] = &internalTask{
|
|
|
|
state: indexTaskPrepare,
|
|
|
|
segmentInfo: nil,
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
return
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
logutil.Logger(fsw.ctx).Info("flushedSegmentWatcher already have the task", zap.Int64("segmentID", segmentID))
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) internalScheduler() {
|
|
|
|
log.Info("IndexCoord flushedSegmentWatcher internalScheduler start...")
|
|
|
|
defer fsw.wg.Done()
|
|
|
|
|
|
|
|
ticker := time.NewTicker(fsw.scheduleDuration)
|
|
|
|
defer ticker.Stop()
|
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-fsw.ctx.Done():
|
|
|
|
log.Warn("IndexCoord flushedSegmentWatcher context done")
|
|
|
|
return
|
|
|
|
case <-ticker.C:
|
|
|
|
fsw.internalRun()
|
|
|
|
case <-fsw.internalNotify:
|
|
|
|
fsw.internalRun()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) internalRun() {
|
2022-09-09 07:52:35 +00:00
|
|
|
fsw.internalTaskMutex.RLock()
|
|
|
|
segmentIDs := make([]UniqueID, 0, len(fsw.internalTasks))
|
2022-08-25 07:48:54 +00:00
|
|
|
if len(fsw.internalTasks) > 0 {
|
|
|
|
log.Debug("IndexCoord flushedSegmentWatcher schedule internal tasks", zap.Int("internal task num", len(fsw.internalTasks)))
|
2022-09-09 07:52:35 +00:00
|
|
|
for segID := range fsw.internalTasks {
|
|
|
|
segmentIDs = append(segmentIDs, segID)
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
2022-09-09 07:52:35 +00:00
|
|
|
sort.Slice(segmentIDs, func(i, j int) bool {
|
|
|
|
return segmentIDs[i] < segmentIDs[j]
|
|
|
|
})
|
|
|
|
}
|
|
|
|
fsw.internalTaskMutex.RUnlock()
|
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
for _, segmentID := range segmentIDs {
|
|
|
|
fsw.internalProcess(segmentID)
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-09-09 07:52:35 +00:00
|
|
|
func (fsw *flushedSegmentWatcher) internalNotifyFunc() {
|
|
|
|
select {
|
|
|
|
case fsw.internalNotify <- struct{}{}:
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) updateInternalTaskState(segID UniqueID, state indexTaskState) {
|
|
|
|
fsw.internalTaskMutex.Lock()
|
|
|
|
defer fsw.internalTaskMutex.Unlock()
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Debug("flushedSegmentWatcher updateInternalTaskState", zap.Int64("segID", segID), zap.String("state", state.String()))
|
2022-09-09 07:52:35 +00:00
|
|
|
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)
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Debug("flushedSegmentWatcher delete the internal task", zap.Int64("segID", segID))
|
2022-09-09 07:52:35 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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
|
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Debug("flushedSegmentWatcher set internal task segment info success", zap.Int64("segID", segID))
|
2022-09-09 07:52:35 +00:00
|
|
|
}
|
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
func (fsw *flushedSegmentWatcher) allParentsDone(segIDs []UniqueID) bool {
|
|
|
|
fsw.internalTaskMutex.RLock()
|
|
|
|
defer fsw.internalTaskMutex.RUnlock()
|
|
|
|
done := true
|
|
|
|
for _, segID := range segIDs {
|
|
|
|
if _, ok := fsw.internalTasks[segID]; ok {
|
|
|
|
done = false
|
|
|
|
break
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
return done
|
2022-09-09 07:52:35 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) internalProcess(segID UniqueID) {
|
|
|
|
t := fsw.getInternalTask(segID)
|
|
|
|
log.Debug("IndexCoord flushedSegmentWatcher process internal task", zap.Int64("segID", segID),
|
2022-08-25 07:48:54 +00:00
|
|
|
zap.String("state", t.state.String()))
|
2022-09-09 07:52:35 +00:00
|
|
|
|
2022-08-25 07:48:54 +00:00
|
|
|
switch t.state {
|
2022-09-15 07:44:31 +00:00
|
|
|
case indexTaskPrepare:
|
|
|
|
if err := fsw.prepare(segID); err != nil {
|
|
|
|
log.Error("flushedSegmentWatcher prepare internal task fail", zap.Int64("segID", segID), zap.Error(err))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
fsw.updateInternalTaskState(segID, indexTaskInit)
|
2022-08-25 07:48:54 +00:00
|
|
|
case indexTaskInit:
|
2022-09-15 07:44:31 +00:00
|
|
|
if err := fsw.constructTask(t); err != nil {
|
|
|
|
log.Error("flushedSegmentWatcher construct task fail", zap.Int64("segID", segID), zap.Error(err))
|
|
|
|
return
|
|
|
|
}
|
2022-09-09 07:52:35 +00:00
|
|
|
fsw.updateInternalTaskState(segID, indexTaskInProgress)
|
|
|
|
fsw.internalNotifyFunc()
|
2022-08-25 07:48:54 +00:00
|
|
|
case indexTaskInProgress:
|
2022-09-15 07:44:31 +00:00
|
|
|
state := fsw.meta.GetSegmentIndexState(segID)
|
|
|
|
if state.state == commonpb.IndexState_Finished || state.state == commonpb.IndexState_Failed || state.state == commonpb.IndexState_IndexStateNone {
|
|
|
|
log.Debug("all tasks are finished", zap.Int64("segID", segID), zap.String("state", state.state.String()))
|
2022-09-09 07:52:35 +00:00
|
|
|
fsw.updateInternalTaskState(segID, indexTaskDone)
|
|
|
|
fsw.internalNotifyFunc()
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
case indexTaskDone:
|
2022-09-15 07:44:31 +00:00
|
|
|
if !fsw.allParentsDone(t.segmentInfo.CompactionFrom) {
|
|
|
|
log.Debug("flushed segment create index done, but there are still parent task that haven't written handoff event",
|
|
|
|
zap.Int64("segID", segID), zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
indexInfos := fsw.meta.GetSegmentIndexes(segID)
|
|
|
|
enableIndex := len(indexInfos) > 0
|
2022-08-25 07:48:54 +00:00
|
|
|
handoffTask := &querypb.SegmentInfo{
|
2022-09-09 07:52:35 +00:00
|
|
|
SegmentID: segID,
|
2022-08-25 07:48:54 +00:00
|
|
|
CollectionID: t.segmentInfo.CollectionID,
|
|
|
|
PartitionID: t.segmentInfo.PartitionID,
|
|
|
|
NumRows: t.segmentInfo.NumOfRows,
|
2022-08-31 03:28:58 +00:00
|
|
|
DmChannel: t.segmentInfo.GetInsertChannel(),
|
2022-08-25 07:48:54 +00:00
|
|
|
IndexName: "",
|
|
|
|
IndexID: 0,
|
|
|
|
CompactionFrom: t.segmentInfo.CompactionFrom,
|
|
|
|
CreatedByCompaction: t.segmentInfo.CreatedByCompaction,
|
|
|
|
SegmentState: t.segmentInfo.State,
|
2022-09-15 07:44:31 +00:00
|
|
|
IndexInfos: make([]*querypb.FieldIndexInfo, 0),
|
|
|
|
EnableIndex: enableIndex,
|
|
|
|
}
|
|
|
|
for _, indexInfo := range indexInfos {
|
|
|
|
handoffTask.IndexInfos = append(handoffTask.IndexInfos, &querypb.FieldIndexInfo{
|
|
|
|
FieldID: fsw.meta.GetFieldIDByIndexID(t.segmentInfo.CollectionID, indexInfo.IndexID),
|
|
|
|
EnableIndex: true,
|
|
|
|
IndexName: fsw.meta.GetIndexNameByID(t.segmentInfo.CollectionID, indexInfo.IndexID),
|
|
|
|
IndexID: indexInfo.IndexID,
|
|
|
|
BuildID: indexInfo.BuildID,
|
|
|
|
IndexParams: fsw.meta.GetIndexParams(t.segmentInfo.CollectionID, indexInfo.IndexID),
|
|
|
|
//IndexFilePaths: nil,
|
|
|
|
//IndexSize: 0,
|
|
|
|
})
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
|
2022-08-25 07:48:54 +00:00
|
|
|
if err := fsw.writeHandoffSegment(handoffTask); err != nil {
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Error("IndexCoord flushSegmentWatcher writeHandoffSegment with no index info fail",
|
2022-09-09 07:52:35 +00:00
|
|
|
zap.Int64("segID", segID), zap.Error(err))
|
2022-08-25 07:48:54 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
if err := fsw.removeFlushedSegment(t); err != nil {
|
|
|
|
log.Error("IndexCoord flushSegmentWatcher removeFlushedSegment fail",
|
2022-09-09 07:52:35 +00:00
|
|
|
zap.Int64("segID", segID), zap.Error(err))
|
2022-08-25 07:48:54 +00:00
|
|
|
return
|
|
|
|
}
|
2022-09-09 07:52:35 +00:00
|
|
|
fsw.deleteInternalTask(segID)
|
|
|
|
fsw.internalNotifyFunc()
|
2022-08-25 07:48:54 +00:00
|
|
|
default:
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Debug("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", segID),
|
2022-08-25 07:48:54 +00:00
|
|
|
zap.String("state", t.state.String()))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
func (fsw *flushedSegmentWatcher) constructTask(t *internalTask) error {
|
|
|
|
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 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
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, index := range fieldIndexes {
|
2022-08-25 07:48:54 +00:00
|
|
|
segIdx := &model.SegmentIndex{
|
2022-09-15 07:44:31 +00:00
|
|
|
SegmentID: t.segmentInfo.ID,
|
|
|
|
CollectionID: t.segmentInfo.CollectionID,
|
|
|
|
PartitionID: t.segmentInfo.PartitionID,
|
|
|
|
NumRows: t.segmentInfo.NumOfRows,
|
|
|
|
IndexID: index.IndexID,
|
|
|
|
CreateTime: t.segmentInfo.StartPosition.Timestamp,
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
//create index task for metaTable
|
|
|
|
// send to indexBuilder
|
|
|
|
have, buildID, err := fsw.ic.createIndexForSegment(segIdx)
|
|
|
|
if err != nil {
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Warn("IndexCoord create index for segment fail", zap.Int64("segID", t.segmentInfo.ID),
|
|
|
|
zap.Int64("indexID", index.IndexID), zap.Error(err))
|
|
|
|
return err
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
if !have {
|
|
|
|
fsw.builder.enqueue(buildID)
|
|
|
|
}
|
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
log.Debug("flushedSegmentWatcher construct children task success", zap.Int64("segID", t.segmentInfo.ID),
|
|
|
|
zap.Int("tasks num", len(fieldIndexes)))
|
|
|
|
return nil
|
2022-08-25 07:48:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) writeHandoffSegment(t *querypb.SegmentInfo) error {
|
|
|
|
key := fmt.Sprintf("%s/%d/%d/%d", util.HandoffSegmentPrefix, t.CollectionID, t.PartitionID, t.SegmentID)
|
|
|
|
value, err := proto.Marshal(t)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("IndexCoord marshal handoff task fail", zap.Int64("collID", t.CollectionID),
|
|
|
|
zap.Int64("partID", t.PartitionID), zap.Int64("segID", t.SegmentID), zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
err = fsw.kvClient.Save(key, string(value))
|
|
|
|
if err != nil {
|
|
|
|
log.Error("IndexCoord save handoff task fail", zap.Int64("collID", t.CollectionID),
|
|
|
|
zap.Int64("partID", t.PartitionID), zap.Int64("segID", t.SegmentID), zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Info("IndexCoord write handoff task success", zap.Int64("collID", t.CollectionID),
|
|
|
|
zap.Int64("partID", t.PartitionID), zap.Int64("segID", t.SegmentID))
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (fsw *flushedSegmentWatcher) removeFlushedSegment(t *internalTask) error {
|
|
|
|
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 {
|
|
|
|
log.Error("IndexCoord remove flushed segment fail", zap.Int64("collID", t.segmentInfo.CollectionID),
|
|
|
|
zap.Int64("partID", t.segmentInfo.PartitionID), zap.Int64("segID", t.segmentInfo.ID), zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
log.Info("IndexCoord remove flushed segment success", zap.Int64("collID", t.segmentInfo.CollectionID),
|
|
|
|
zap.Int64("partID", t.segmentInfo.PartitionID), zap.Int64("segID", t.segmentInfo.ID))
|
|
|
|
return nil
|
|
|
|
}
|
2022-09-09 07:52:35 +00:00
|
|
|
|
|
|
|
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)
|
|
|
|
}
|
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
func (fsw *flushedSegmentWatcher) prepare(segID UniqueID) error {
|
|
|
|
defer fsw.internalNotifyFunc()
|
|
|
|
log.Debug("prepare flushed segment task", zap.Int64("segID", segID))
|
2022-09-09 07:52:35 +00:00
|
|
|
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)
|
2022-09-15 07:44:31 +00:00
|
|
|
return err
|
2022-09-09 07:52:35 +00:00
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
return err
|
2022-09-09 07:52:35 +00:00
|
|
|
}
|
2022-09-15 07:44:31 +00:00
|
|
|
//t := fsw.getInternalTask(segID)
|
|
|
|
//if t.segmentInfo.CreatedByCompaction {
|
|
|
|
// if err := fsw.removeCompactedTasks(t); err != nil {
|
|
|
|
// return err
|
|
|
|
// }
|
|
|
|
//}
|
|
|
|
return nil
|
2022-09-09 07:52:35 +00:00
|
|
|
}
|
|
|
|
|
2022-09-15 07:44:31 +00:00
|
|
|
//func (fsw *flushedSegmentWatcher) removeCompactedTasks(t *internalTask) error {
|
|
|
|
// log.Debug("IndexCoord flushedSegmentWatcher mark task as deleted which is compacted", zap.Int64("segID", t.segmentInfo.ID),
|
|
|
|
// zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom))
|
|
|
|
// if err := fsw.builder.markTasksAsDeleted(fsw.meta.GetBuildIDsFromSegIDs(t.segmentInfo.CompactionFrom)); err != nil {
|
|
|
|
// log.Error("mark index meta fail, try again", zap.Int64s("compacted segIDs", t.segmentInfo.CompactionFrom),
|
|
|
|
// zap.Error(err))
|
|
|
|
// return err
|
|
|
|
// }
|
|
|
|
// for _, segID := range t.segmentInfo.CompactionFrom {
|
|
|
|
// fsw.deleteChildrenTask(segID)
|
|
|
|
// if _, ok := fsw.internalTasks[segID]; ok {
|
|
|
|
// fsw.updateInternalTaskState(segID, indexTaskDeleted)
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
// return nil
|
|
|
|
//}
|