Merge IndexCoord and DataCoord (#21267)

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
This commit is contained in:
cai.zhang 2023-01-04 19:37:36 +08:00 committed by GitHub
parent 7b39873ae0
commit e5f408dceb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
92 changed files with 11139 additions and 3789 deletions

View File

@ -110,7 +110,7 @@ func (c *run) formatFlags(args []string, flags *flag.FlagSet) {
flags.BoolVar(&c.enableRootCoord, typeutil.RootCoordRole, false, "enable root coordinator")
flags.BoolVar(&c.enableQueryCoord, typeutil.QueryCoordRole, false, "enable query coordinator")
flags.BoolVar(&c.enableIndexCoord, typeutil.IndexCoordRole, false, "enable index coordinator")
//flags.BoolVar(&c.enableIndexCoord, typeutil.IndexCoordRole, false, "enable index coordinator")
flags.BoolVar(&c.enableDataCoord, typeutil.DataCoordRole, false, "enable data coordinator")
flags.BoolVar(&c.enableQueryNode, typeutil.QueryNodeRole, false, "enable query node")

View File

@ -4,7 +4,7 @@ import (
"github.com/blang/semver/v4"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/cmd/tools/migration/versions"
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
"github.com/milvus-io/milvus/internal/metastore/kv/querycoord"
"github.com/milvus-io/milvus/internal/metastore/kv/rootcoord"
"github.com/milvus-io/milvus/internal/metastore/model"
@ -188,7 +188,7 @@ func (meta *CollectionIndexesMeta220) GenerateSaves() (map[string]string, error)
for collectionID := range *meta {
for indexID := range (*meta)[collectionID] {
ckey := indexcoord.BuildIndexKey(collectionID, indexID)
ckey := datacoord.BuildIndexKey(collectionID, indexID)
index := (*meta)[collectionID][indexID]
var value string
indexPb := model.MarshalIndexModel(index)
@ -210,7 +210,7 @@ func (meta *SegmentIndexesMeta220) GenerateSaves() (map[string]string, error) {
for segmentID := range *meta {
for indexID := range (*meta)[segmentID] {
index := (*meta)[segmentID][indexID]
ckey := indexcoord.BuildSegmentIndexKey(index.CollectionID, index.PartitionID, index.SegmentID, index.BuildID)
ckey := datacoord.BuildSegmentIndexKey(index.CollectionID, index.PartitionID, index.SegmentID, index.BuildID)
var value string
indexPb := model.MarshalSegmentIndexModel(index)
marshaledIndexPb, err := proto.Marshal(indexPb)

View File

@ -298,6 +298,7 @@ dataCoord:
# over (compactableProportion * segment max # of rows) rows.
# MUST BE GREATER THAN OR EQUAL TO <smallProportion>!!!
expansionRate: 1.25 # During compaction, the size of segment # of rows is able to exceed segment max # of rows by (expansionRate-1) * 100%.
minSegmentNumRowsToEnableIndex: 1024 # It's a threshold. When the segment num rows is less than this value, the segment will not be indexed
compaction:
enableAutoCompaction: true
@ -307,6 +308,13 @@ dataCoord:
missingTolerance: 86400 # file meta missing tolerance duration in seconds, 60*24
dropTolerance: 86400 # file belongs to dropped entity tolerance duration in seconds, 60*24
bindIndexNodeMode:
enable: false
address: "localhost:22930"
withCred: false
nodeID: 0
dataNode:
port: 21124

View File

@ -0,0 +1,28 @@
// 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 datacoord
import "sort"
type buildIndexPolicy func(buildIDs []UniqueID)
func defaultBuildIndexPolicy(buildIDs []UniqueID) {
sort.Slice(buildIDs, func(i, j int) bool {
return buildIDs[i] < buildIDs[j]
})
}

View File

@ -100,20 +100,20 @@ type compactionPlanHandler struct {
quit chan struct{}
wg sync.WaitGroup
flushCh chan UniqueID
segRefer *SegmentReferenceManager
parallelCh map[int64]chan struct{}
//segRefer *SegmentReferenceManager
parallelCh map[int64]chan struct{}
}
func newCompactionPlanHandler(sessions *SessionManager, cm *ChannelManager, meta *meta,
allocator allocator, flush chan UniqueID, segRefer *SegmentReferenceManager) *compactionPlanHandler {
allocator allocator, flush chan UniqueID) *compactionPlanHandler {
return &compactionPlanHandler{
plans: make(map[int64]*compactionTask),
chManager: cm,
meta: meta,
sessions: sessions,
allocator: allocator,
flushCh: flush,
segRefer: segRefer,
plans: make(map[int64]*compactionTask),
chManager: cm,
meta: meta,
sessions: sessions,
allocator: allocator,
flushCh: flush,
//segRefer: segRefer,
parallelCh: make(map[int64]chan struct{}),
}
}

View File

@ -322,18 +322,12 @@ func TestCompactionPlanHandler_handleMergeCompactionResult(t *testing.T) {
plans: plans,
sessions: sessions,
meta: meta,
segRefer: &SegmentReferenceManager{
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{},
},
}
c2 := &compactionPlanHandler{
plans: plans,
sessions: sessions,
meta: errMeta,
segRefer: &SegmentReferenceManager{
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{},
},
}
compactionResult := &datapb.CompactionResult{
@ -380,9 +374,6 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
t.Run("test not exists compaction task", func(t *testing.T) {
c := &compactionPlanHandler{
plans: map[int64]*compactionTask{1: {}},
segRefer: &SegmentReferenceManager{
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{},
},
}
err := c.completeCompaction(&datapb.CompactionResult{PlanID: 2})
assert.Error(t, err)
@ -390,9 +381,6 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
t.Run("test completed compaction task", func(t *testing.T) {
c := &compactionPlanHandler{
plans: map[int64]*compactionTask{1: {state: completed}},
segRefer: &SegmentReferenceManager{
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{},
},
}
err := c.completeCompaction(&datapb.CompactionResult{PlanID: 1})
assert.Error(t, err)
@ -480,9 +468,6 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
sessions: sessions,
meta: meta,
flushCh: flushCh,
segRefer: &SegmentReferenceManager{
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{},
},
}
err := c.completeCompaction(&compactionResult)
@ -581,9 +566,6 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
sessions: sessions,
meta: meta,
flushCh: flushCh,
segRefer: &SegmentReferenceManager{
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{},
},
}
err := c.completeCompaction(&compactionResult)
@ -780,7 +762,6 @@ func Test_newCompactionPlanHandler(t *testing.T) {
meta *meta
allocator allocator
flush chan UniqueID
segRefer *SegmentReferenceManager
}
tests := []struct {
name string
@ -795,7 +776,6 @@ func Test_newCompactionPlanHandler(t *testing.T) {
&meta{},
newMockAllocator(),
nil,
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
},
&compactionPlanHandler{
plans: map[int64]*compactionTask{},
@ -804,14 +784,13 @@ func Test_newCompactionPlanHandler(t *testing.T) {
meta: &meta{},
allocator: newMockAllocator(),
flushCh: nil,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
parallelCh: make(map[int64]chan struct{}),
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := newCompactionPlanHandler(tt.args.sessions, tt.args.cm, tt.args.meta, tt.args.allocator, tt.args.flush, tt.args.segRefer)
got := newCompactionPlanHandler(tt.args.sessions, tt.args.cm, tt.args.meta, tt.args.allocator, tt.args.flush)
assert.EqualValues(t, tt.want, got)
})
}

View File

@ -23,17 +23,15 @@ import (
"sync"
"time"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/indexparamcheck"
"github.com/milvus-io/milvus/internal/util/logutil"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/samber/lo"
"go.uber.org/zap"
)
type compactTime struct {
@ -66,17 +64,17 @@ type compactionSignal struct {
var _ trigger = (*compactionTrigger)(nil)
type compactionTrigger struct {
handler Handler
meta *meta
allocator allocator
signals chan *compactionSignal
compactionHandler compactionPlanContext
globalTrigger *time.Ticker
forceMu sync.Mutex
quit chan struct{}
wg sync.WaitGroup
segRefer *SegmentReferenceManager
indexCoord types.IndexCoord
handler Handler
meta *meta
allocator allocator
signals chan *compactionSignal
compactionHandler compactionPlanContext
globalTrigger *time.Ticker
forceMu sync.Mutex
quit chan struct{}
wg sync.WaitGroup
//segRefer *SegmentReferenceManager
//indexCoord types.IndexCoord
estimateNonDiskSegmentPolicy calUpperLimitPolicy
estimateDiskSegmentPolicy calUpperLimitPolicy
// A sloopy hack, so we can test with different segment row count without worrying that
@ -88,17 +86,17 @@ func newCompactionTrigger(
meta *meta,
compactionHandler compactionPlanContext,
allocator allocator,
segRefer *SegmentReferenceManager,
indexCoord types.IndexCoord,
//segRefer *SegmentReferenceManager,
//indexCoord types.IndexCoord,
handler Handler,
) *compactionTrigger {
return &compactionTrigger{
meta: meta,
allocator: allocator,
signals: make(chan *compactionSignal, 100),
compactionHandler: compactionHandler,
segRefer: segRefer,
indexCoord: indexCoord,
meta: meta,
allocator: allocator,
signals: make(chan *compactionSignal, 100),
compactionHandler: compactionHandler,
//segRefer: segRefer,
//indexCoord: indexCoord,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
handler: handler,
@ -280,39 +278,29 @@ func (t *compactionTrigger) reCalcSegmentMaxNumOfRows(collectionID UniqueID, isD
// TODO: Update segment info should be written back to Etcd.
func (t *compactionTrigger) updateSegmentMaxSize(segments []*SegmentInfo) (bool, error) {
ctx := context.Background()
if len(segments) == 0 {
return false, nil
}
collectionID := segments[0].GetCollectionID()
resp, err := t.indexCoord.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{
CollectionID: collectionID,
IndexName: "",
})
if err != nil {
return false, err
}
indexInfos := t.meta.GetIndexesForCollection(segments[0].GetCollectionID(), "")
isDiskANN := false
for _, indexInfo := range resp.IndexInfos {
indexParamsMap := funcutil.KeyValuePair2Map(indexInfo.IndexParams)
if indexType, ok := indexParamsMap["index_type"]; ok {
if indexType == indexparamcheck.IndexDISKANN {
// If index type is DiskANN, recalc segment max size here.
isDiskANN = true
newMaxRows, err := t.reCalcSegmentMaxNumOfRows(collectionID, true)
if err != nil {
return false, err
}
if len(segments) > 0 && int64(newMaxRows) != segments[0].GetMaxRowNum() {
log.Info("segment max rows recalculated for DiskANN collection",
zap.Int64("old max rows", segments[0].GetMaxRowNum()),
zap.Int64("new max rows", int64(newMaxRows)))
for _, segment := range segments {
segment.MaxRowNum = int64(newMaxRows)
}
for _, indexInfo := range indexInfos {
indexType := getIndexType(indexInfo.IndexParams)
if indexType == indexparamcheck.IndexDISKANN {
// If index type is DiskANN, recalc segment max size here.
isDiskANN = true
newMaxRows, err := t.reCalcSegmentMaxNumOfRows(collectionID, true)
if err != nil {
return false, err
}
if len(segments) > 0 && int64(newMaxRows) != segments[0].GetMaxRowNum() {
log.Info("segment max rows recalculated for DiskANN collection",
zap.Int64("old max rows", segments[0].GetMaxRowNum()),
zap.Int64("new max rows", int64(newMaxRows)))
for _, segment := range segments {
segment.MaxRowNum = int64(newMaxRows)
}
}
}
@ -365,7 +353,7 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) {
break
}
group.segments = FilterInIndexedSegments(t.handler, t.indexCoord, group.segments...)
//group.segments = FilterInIndexedSegments(t.handler, t.meta, group.segments...)
isDiskIndex, err := t.updateSegmentMaxSize(group.segments)
if err != nil {
@ -734,7 +722,7 @@ func reverseGreedySelect(candidates []*SegmentInfo, free int64, maxSegment int)
func (t *compactionTrigger) getCandidateSegments(channel string, partitionID UniqueID) []*SegmentInfo {
segments := t.meta.GetSegmentsByChannel(channel)
segments = FilterInIndexedSegments(t.handler, t.indexCoord, segments...)
segments = FilterInIndexedSegments(t.handler, t.meta, segments...)
var res []*SegmentInfo
for _, s := range segments {
if !isSegmentHealthy(s) ||

View File

@ -22,15 +22,15 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/tsoutil"
)
type spyCompactionHandler struct {
@ -82,7 +82,6 @@ func Test_compactionTrigger_force(t *testing.T) {
signals chan *compactionSignal
compactionHandler compactionPlanContext
globalTrigger *time.Ticker
segRefer *SegmentReferenceManager
}
paramtable.Init()
@ -94,6 +93,7 @@ func Test_compactionTrigger_force(t *testing.T) {
timeTravel := tsoutil.ComposeTS(ttRetention.UnixNano()/int64(time.Millisecond), 0)
vecFieldID := int64(201)
indexID := int64(1001)
tests := []struct {
name string
fields fields
@ -132,6 +132,25 @@ func Test_compactionTrigger_force(t *testing.T) {
},
},
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: 1,
CollectionID: 2,
PartitionID: 1,
NumRows: 100,
IndexID: indexID,
BuildID: 1,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
},
2: {
SegmentInfo: &datapb.SegmentInfo{
@ -158,6 +177,25 @@ func Test_compactionTrigger_force(t *testing.T) {
},
},
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: 2,
CollectionID: 2,
PartitionID: 1,
NumRows: 100,
IndexID: indexID,
BuildID: 2,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
},
3: {
SegmentInfo: &datapb.SegmentInfo{
@ -170,6 +208,25 @@ func Test_compactionTrigger_force(t *testing.T) {
InsertChannel: "ch1",
State: commonpb.SegmentState_Flushed,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: 3,
CollectionID: 1111,
PartitionID: 1,
NumRows: 100,
IndexID: indexID,
BuildID: 3,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
},
},
},
@ -294,12 +351,53 @@ func Test_compactionTrigger_force(t *testing.T) {
},
},
},
indexes: map[UniqueID]map[UniqueID]*model.Index{
2: {
indexID: {
TenantID: "",
CollectionID: 2,
FieldID: vecFieldID,
IndexID: indexID,
IndexName: "_default_idx",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: []*commonpb.KeyValuePair{
{
Key: "index_type",
Value: "HNSW",
},
},
IsAutoIndex: false,
UserIndexParams: nil,
},
},
1000: {
indexID: {
TenantID: "",
CollectionID: 1000,
FieldID: vecFieldID,
IndexID: indexID,
IndexName: "_default_idx",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: []*commonpb.KeyValuePair{
{
Key: "index_type",
Value: "DISKANN",
},
},
IsAutoIndex: false,
UserIndexParams: nil,
},
},
},
},
&MockAllocator0{},
nil,
&spyCompactionHandler{spyChan: make(chan *datapb.CompactionPlan, 1)},
nil,
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
},
2,
false,
@ -356,8 +454,6 @@ func Test_compactionTrigger_force(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -365,8 +461,6 @@ func Test_compactionTrigger_force(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCoord,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -380,7 +474,6 @@ func Test_compactionTrigger_force(t *testing.T) {
})
t.Run(tt.name+" with DiskANN index", func(t *testing.T) {
indexCood := newMockIndexCoord()
segmentIDs := make([]int64, 0)
for _, segment := range tt.fields.meta.segments.GetSegments() {
segmentIDs = append(segmentIDs, segment.GetID())
@ -394,8 +487,6 @@ func Test_compactionTrigger_force(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -409,147 +500,8 @@ func Test_compactionTrigger_force(t *testing.T) {
<-spy.spyChan
})
t.Run(tt.name+" with no filtered segments", func(t *testing.T) {
indexCood := newMockIndexCoord()
segmentIDs := make([]int64, 0)
for _, segment := range tt.fields.meta.segments.GetSegments() {
segmentIDs = append(segmentIDs, segment.GetID())
// Collection 2000 means has no vector field
segment.CollectionID = 2000
}
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
allocator: tt.fields.allocator,
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
}
tt.collectionID = 2000
_, err := tr.forceTriggerCompaction(tt.collectionID)
assert.Equal(t, tt.wantErr, err != nil)
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
hasPlan := true
select {
case <-spy.spyChan:
hasPlan = true
case <-time.After(2 * time.Second):
hasPlan = false
}
assert.Equal(t, false, hasPlan)
})
t.Run(tt.name+" with meta error", func(t *testing.T) {
indexCood := newMockIndexCoord()
segmentIDs := make([]int64, 0)
for _, segment := range tt.fields.meta.segments.GetSegments() {
segmentIDs = append(segmentIDs, segment.GetID())
// Collection 3000 means field has no dim info
segment.CollectionID = 3000
}
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
allocator: tt.fields.allocator,
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
}
tt.collectionID = 3000
_, err := tr.forceTriggerCompaction(tt.collectionID)
assert.Equal(t, tt.wantErr, err != nil)
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
hasPlan := true
select {
case <-spy.spyChan:
hasPlan = true
case <-time.After(2 * time.Second):
hasPlan = false
}
assert.Equal(t, false, hasPlan)
})
t.Run(tt.name+" with meta error", func(t *testing.T) {
indexCood := newMockIndexCoord()
segmentIDs := make([]int64, 0)
for _, segment := range tt.fields.meta.segments.GetSegments() {
segmentIDs = append(segmentIDs, segment.GetID())
// Collection 4000 means field has dim mistake
segment.CollectionID = 4000
}
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
allocator: tt.fields.allocator,
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
}
tt.collectionID = 4000
_, err := tr.forceTriggerCompaction(tt.collectionID)
assert.Equal(t, tt.wantErr, err != nil)
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
hasPlan := true
select {
case <-spy.spyChan:
hasPlan = true
case <-time.After(2 * time.Second):
hasPlan = false
}
assert.Equal(t, false, hasPlan)
})
t.Run(tt.name+" with index info error", func(t *testing.T) {
indexCood := newMockIndexCoord()
segmentIDs := make([]int64, 0)
for _, segment := range tt.fields.meta.segments.GetSegments() {
segmentIDs = append(segmentIDs, segment.GetID())
// Collection 10000 means it has DescribeIndexInfo error
segment.CollectionID = 10000
}
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
allocator: tt.fields.allocator,
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
testingOnly: true,
}
tt.collectionID = 10000
_, err := tr.forceTriggerCompaction(tt.collectionID)
assert.Equal(t, tt.wantErr, err != nil)
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
hasPlan := true
select {
case <-spy.spyChan:
hasPlan = true
case <-time.After(2 * time.Second):
hasPlan = false
}
assert.Equal(t, false, hasPlan)
})
t.Run(tt.name+" with allocate ts error", func(t *testing.T) {
indexCood := newMockIndexCoord()
//indexCood := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -557,8 +509,6 @@ func Test_compactionTrigger_force(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -608,7 +558,6 @@ func Test_compactionTrigger_force(t *testing.T) {
})
t.Run(tt.name+" with getCompact error", func(t *testing.T) {
indexCood := newMockIndexCoord()
for _, segment := range tt.fields.meta.segments.GetSegments() {
segment.CollectionID = 1111
}
@ -619,8 +568,6 @@ func Test_compactionTrigger_force(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: tt.fields.segRefer,
indexCoord: indexCood,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -813,8 +760,6 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -822,8 +767,6 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
indexCoord: indexCoord,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -962,7 +905,6 @@ func Test_compactionTrigger_noplan(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
@ -971,8 +913,6 @@ func Test_compactionTrigger_noplan(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
indexCoord: indexCoord,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -1118,8 +1058,6 @@ func Test_compactionTrigger_PrioritizedCandi(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -1127,8 +1065,6 @@ func Test_compactionTrigger_PrioritizedCandi(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
indexCoord: indexCoord,
testingOnly: true,
}
tr.start()
@ -1257,8 +1193,6 @@ func Test_compactionTrigger_SmallCandi(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -1266,8 +1200,6 @@ func Test_compactionTrigger_SmallCandi(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
indexCoord: indexCoord,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -1395,8 +1327,6 @@ func Test_compactionTrigger_SqueezeNonPlannedSegs(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -1404,8 +1334,6 @@ func Test_compactionTrigger_SqueezeNonPlannedSegs(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
indexCoord: indexCoord,
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
@ -1529,8 +1457,6 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
tr := &compactionTrigger{
meta: tt.fields.meta,
handler: newMockHandlerWithMeta(tt.fields.meta),
@ -1538,8 +1464,6 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
signals: tt.fields.signals,
compactionHandler: tt.fields.compactionHandler,
globalTrigger: tt.fields.globalTrigger,
segRefer: &SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}},
indexCoord: indexCoord,
testingOnly: true,
}
tr.start()
@ -1585,9 +1509,7 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
func Test_compactionTrigger_shouldDoSingleCompaction(t *testing.T) {
Params.Init()
indexCoord := newMockIndexCoord()
trigger := newCompactionTrigger(&meta{}, &compactionPlanHandler{}, newMockAllocator(),
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, indexCoord, newMockHandler())
trigger := newCompactionTrigger(&meta{}, &compactionPlanHandler{}, newMockAllocator(), newMockHandler())
// Test too many deltalogs.
var binlogs []*datapb.FieldBinlog
@ -1752,9 +1674,7 @@ func Test_newCompactionTrigger(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
indexCoord := newMockIndexCoord()
got := newCompactionTrigger(tt.args.meta, tt.args.compactionHandler, tt.args.allocator,
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, indexCoord, newMockHandler())
got := newCompactionTrigger(tt.args.meta, tt.args.compactionHandler, tt.args.allocator, newMockHandler())
assert.Equal(t, tt.args.meta, got.meta)
assert.Equal(t, tt.args.compactionHandler, got.compactionHandler)
assert.Equal(t, tt.args.allocator, got.allocator)
@ -1763,10 +1683,7 @@ func Test_newCompactionTrigger(t *testing.T) {
}
func Test_handleSignal(t *testing.T) {
indexCoord := newMockIndexCoord()
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, newMockAllocator(),
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, indexCoord, newMockHandler())
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, newMockAllocator(), newMockHandler())
signal := &compactionSignal{
segmentID: 1,
}
@ -1776,14 +1693,12 @@ func Test_handleSignal(t *testing.T) {
}
func Test_allocTs(t *testing.T) {
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, newMockAllocator(),
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, nil, newMockHandler())
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, newMockAllocator(), newMockHandler())
ts, err := got.allocTs()
assert.NoError(t, err)
assert.True(t, ts > 0)
got = newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, &FailsAllocator{},
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, nil, newMockHandler())
got = newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, &FailsAllocator{}, newMockHandler())
ts, err = got.allocTs()
assert.Error(t, err)
assert.Equal(t, uint64(0), ts)
@ -1811,7 +1726,7 @@ func Test_getCompactTime(t *testing.T) {
m := &meta{segments: NewSegmentsInfo(), collections: collections}
got := newCompactionTrigger(m, &compactionPlanHandler{}, newMockAllocator(),
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, nil, &ServerHandler{
&ServerHandler{
&Server{
meta: m,
},

View File

@ -25,3 +25,10 @@ const (
const (
moduleName = "DataCoord"
)
const (
flatIndex = "FLAT"
binFlatIndex = "BIN_FLAT"
diskAnnIndex = "DISKANN"
invalidIndex = "invalid"
)

View File

@ -43,3 +43,7 @@ func errDataCoordIsUnhealthy(coordID UniqueID) error {
func msgSegmentNotFound(segID UniqueID) string {
return fmt.Sprintf("failed to get segment %d", segID)
}
func msgAmbiguousIndexName() string {
return "there are multiple indexes, please specify the index_name"
}

View File

@ -23,15 +23,17 @@ import (
"sync"
"time"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/minio/minio-go/v7"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
const (
@ -53,11 +55,9 @@ type GcOption struct {
// garbageCollector handles garbage files in object storage
// which could be dropped collection remanent or data node failure traces
type garbageCollector struct {
option GcOption
meta *meta
handler Handler
segRefer *SegmentReferenceManager
indexCoord types.IndexCoord
option GcOption
meta *meta
handler Handler
startOnce sync.Once
stopOnce sync.Once
@ -66,16 +66,14 @@ type garbageCollector struct {
}
// newGarbageCollector create garbage collector with meta and option
func newGarbageCollector(meta *meta, handler Handler, segRefer *SegmentReferenceManager, indexCoord types.IndexCoord, opt GcOption) *garbageCollector {
func newGarbageCollector(meta *meta, handler Handler, opt GcOption) *garbageCollector {
log.Info("GC with option", zap.Bool("enabled", opt.enabled), zap.Duration("interval", opt.checkInterval),
zap.Duration("missingTolerance", opt.missingTolerance), zap.Duration("dropTolerance", opt.dropTolerance))
return &garbageCollector{
meta: meta,
handler: handler,
segRefer: segRefer,
indexCoord: indexCoord,
option: opt,
closeCh: make(chan struct{}),
meta: meta,
handler: handler,
option: opt,
closeCh: make(chan struct{}),
}
}
@ -101,7 +99,10 @@ func (gc *garbageCollector) work() {
select {
case <-ticker:
gc.clearEtcd()
gc.recycleUnusedIndexes()
gc.recycleUnusedSegIndexes()
gc.scan()
gc.recycleUnusedIndexFiles()
case <-gc.closeCh:
log.Warn("garbage collector quit")
return
@ -170,11 +171,6 @@ func (gc *garbageCollector) scan() {
continue
}
if gc.segRefer.HasSegmentLock(segmentID) {
valid++
continue
}
if strings.Contains(prefix, statsLogPrefix) &&
segmentMap.Contain(segmentID) {
valid++
@ -207,9 +203,10 @@ func (gc *garbageCollector) clearEtcd() {
drops := make(map[int64]*SegmentInfo, 0)
compactTo := make(map[int64]*SegmentInfo)
for _, segment := range all {
if segment.GetState() == commonpb.SegmentState_Dropped && !gc.segRefer.HasSegmentLock(segment.ID) {
if segment.GetState() == commonpb.SegmentState_Dropped {
drops[segment.GetID()] = segment
continue
//continue
// A(indexed), B(indexed) -> C(no indexed), D(no indexed) -> E(no indexed), A, B can not be GC
}
for _, from := range segment.GetCompactionFrom() {
compactTo[from] = segment
@ -222,7 +219,7 @@ func (gc *garbageCollector) clearEtcd() {
droppedCompactTo[to] = struct{}{}
}
}
indexedSegments := FilterInIndexedSegments(gc.handler, gc.indexCoord, lo.Keys(droppedCompactTo)...)
indexedSegments := FilterInIndexedSegments(gc.handler, gc.meta, lo.Keys(droppedCompactTo)...)
indexedSet := make(typeutil.UniqueSet)
for _, segment := range indexedSegments {
indexedSet.Insert(segment.GetID())
@ -287,3 +284,100 @@ func (gc *garbageCollector) removeLogs(logs []*datapb.Binlog) bool {
}
return delFlag
}
func (gc *garbageCollector) recycleUnusedIndexes() {
log.Info("start recycleUnusedIndexes")
deletedIndexes := gc.meta.GetDeletedIndexes()
for _, index := range deletedIndexes {
if err := gc.meta.RemoveIndex(index.CollectionID, index.IndexID); err != nil {
log.Warn("remove index on collection fail", zap.Int64("collID", index.CollectionID),
zap.Int64("indexID", index.IndexID), zap.Error(err))
continue
}
}
}
func (gc *garbageCollector) recycleUnusedSegIndexes() {
segIndexes := gc.meta.GetAllSegIndexes()
for _, segIdx := range segIndexes {
if gc.meta.GetSegmentUnsafe(segIdx.SegmentID) == nil || !gc.meta.IsIndexExist(segIdx.CollectionID, segIdx.IndexID) {
if err := gc.meta.RemoveSegmentIndex(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.IndexID, segIdx.BuildID); err != nil {
log.Warn("delete index meta from etcd failed, wait to retry", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("nodeID", segIdx.NodeID), zap.Error(err))
continue
}
log.Info("index meta recycle success", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("segID", segIdx.SegmentID))
}
}
}
// recycleUnusedIndexFiles is used to delete those index files that no longer exist in the meta.
func (gc *garbageCollector) recycleUnusedIndexFiles() {
log.Info("start recycleUnusedIndexFiles")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
prefix := path.Join(gc.option.cli.RootPath(), common.SegmentIndexPath) + "/"
// list dir first
keys, _, err := gc.option.cli.ListWithPrefix(ctx, prefix, false)
if err != nil {
log.Error("garbageCollector recycleUnusedIndexFiles list keys from chunk manager failed", zap.Error(err))
return
}
for _, key := range keys {
log.Debug("indexFiles keys", zap.String("key", key))
buildID, err := parseBuildIDFromFilePath(key)
if err != nil {
log.Error("garbageCollector recycleUnusedIndexFiles parseIndexFileKey", zap.String("key", key), zap.Error(err))
continue
}
log.Info("garbageCollector will recycle index files", zap.Int64("buildID", buildID))
canRecycle, segIdx := gc.meta.CleanSegmentIndex(buildID)
if !canRecycle {
// Even if the index is marked as deleted, the index file will not be recycled, wait for the next gc,
// and delete all index files about the buildID at one time.
log.Warn("garbageCollector can not recycle index files", zap.Int64("buildID", buildID))
continue
}
if segIdx == nil {
// buildID no longer exists in meta, remove all index files
log.Info("garbageCollector recycleUnusedIndexFiles find meta has not exist, remove index files",
zap.Int64("buildID", buildID))
err = gc.option.cli.RemoveWithPrefix(ctx, key)
if err != nil {
log.Warn("garbageCollector recycleUnusedIndexFiles remove index files failed",
zap.Int64("buildID", buildID), zap.String("prefix", key), zap.Error(err))
continue
}
continue
}
filesMap := make(map[string]struct{})
for _, fileID := range segIdx.IndexFileKeys {
filepath := metautil.BuildSegmentIndexFilePath(gc.option.cli.RootPath(), segIdx.BuildID, segIdx.IndexVersion,
segIdx.PartitionID, segIdx.SegmentID, fileID)
filesMap[filepath] = struct{}{}
}
files, _, err := gc.option.cli.ListWithPrefix(ctx, key, true)
if err != nil {
log.Warn("garbageCollector recycleUnusedIndexFiles list files failed",
zap.Int64("buildID", buildID), zap.String("prefix", key), zap.Error(err))
continue
}
log.Info("recycle index files", zap.Int64("buildID", buildID), zap.Int("meta files num", len(filesMap)),
zap.Int("chunkManager files num", len(files)))
deletedFilesNum := 0
for _, file := range files {
if _, ok := filesMap[file]; !ok {
if err = gc.option.cli.Remove(ctx, file); err != nil {
log.Warn("garbageCollector recycleUnusedIndexFiles remove file failed",
zap.Int64("buildID", buildID), zap.String("file", file), zap.Error(err))
continue
}
deletedFilesNum++
}
}
log.Info("index files recycle success", zap.Int64("buildID", buildID),
zap.Int("delete index files num", deletedFilesNum))
}
}

View File

@ -19,18 +19,28 @@ package datacoord
import (
"bytes"
"context"
"errors"
"path"
"strconv"
"strings"
"sync"
"testing"
"time"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
@ -48,24 +58,8 @@ func Test_garbageCollector_basic(t *testing.T) {
meta, err := newMemoryMeta()
assert.Nil(t, err)
etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
Params.EtcdCfg.Endpoints.GetAsStrings(),
Params.EtcdCfg.EtcdTLSCert.GetValue(),
Params.EtcdCfg.EtcdTLSKey.GetValue(),
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
indexCoord := mocks.NewMockIndexCoord(t)
t.Run("normal gc", func(t *testing.T) {
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Millisecond * 10,
@ -81,7 +75,7 @@ func Test_garbageCollector_basic(t *testing.T) {
})
t.Run("with nil cli", func(t *testing.T) {
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: nil,
enabled: true,
checkInterval: time.Millisecond * 10,
@ -117,61 +111,25 @@ func Test_garbageCollector_scan(t *testing.T) {
meta, err := newMemoryMeta()
assert.Nil(t, err)
etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
Params.EtcdCfg.Endpoints.GetAsStrings(),
Params.EtcdCfg.EtcdTLSCert.GetValue(),
Params.EtcdCfg.EtcdTLSKey.GetValue(),
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
t.Run("key is reference", func(t *testing.T) {
segReferManager := &SegmentReferenceManager{
etcdKV: etcdKV,
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{
1: {
1: {
TaskID: 1,
NodeID: 1,
SegmentIDs: []UniqueID{2},
},
},
},
segmentReferCnt: map[UniqueID]int{
2: 1,
},
}
indexCoord := mocks.NewMockIndexCoord(t)
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Minute * 30,
missingTolerance: time.Hour * 24,
dropTolerance: time.Hour * 24,
})
gc.segRefer = segReferManager
gc.scan()
validateMinioPrefixElements(t, cli.Client, bucketName, path.Join(rootPath, insertLogPrefix), inserts)
validateMinioPrefixElements(t, cli.Client, bucketName, path.Join(rootPath, statsLogPrefix), stats)
validateMinioPrefixElements(t, cli.Client, bucketName, path.Join(rootPath, deltaLogPrefix), delta)
validateMinioPrefixElements(t, cli.Client, bucketName, path.Join(rootPath, `indexes`), others)
err = gc.segRefer.ReleaseSegmentsLock(1, 1)
assert.NoError(t, err)
gc.close()
})
t.Run("missing all but save tolerance", func(t *testing.T) {
indexCoord := mocks.NewMockIndexCoord(t)
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Minute * 30,
@ -196,8 +154,7 @@ func Test_garbageCollector_scan(t *testing.T) {
err = meta.AddSegment(segment)
require.NoError(t, err)
indexCoord := mocks.NewMockIndexCoord(t)
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Minute * 30,
@ -225,8 +182,7 @@ func Test_garbageCollector_scan(t *testing.T) {
err = meta.AddSegment(segment)
require.NoError(t, err)
indexCoord := mocks.NewMockIndexCoord(t)
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Minute * 30,
@ -242,8 +198,7 @@ func Test_garbageCollector_scan(t *testing.T) {
gc.close()
})
t.Run("missing gc all", func(t *testing.T) {
indexCoord := mocks.NewMockIndexCoord(t)
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Minute * 30,
@ -264,8 +219,7 @@ func Test_garbageCollector_scan(t *testing.T) {
})
t.Run("list object with error", func(t *testing.T) {
indexCoord := mocks.NewMockIndexCoord(t)
gc := newGarbageCollector(meta, newMockHandler(), segRefer, indexCoord, GcOption{
gc := newGarbageCollector(meta, newMockHandler(), GcOption{
cli: cli,
enabled: true,
checkInterval: time.Minute * 30,
@ -372,3 +326,656 @@ func cleanupOSS(cli *minio.Client, bucket, root string) {
cli.RemoveObjects(context.TODO(), bucket, ch, minio.RemoveObjectsOptions{})
cli.RemoveBucket(context.TODO(), bucket)
}
func createMetaForRecycleUnusedIndexes(catalog *datacoord.Catalog) *meta {
var (
ctx = context.Background()
collID = UniqueID(100)
//partID = UniqueID(200)
fieldID = UniqueID(300)
indexID = UniqueID(400)
)
return &meta{
RWMutex: sync.RWMutex{},
ctx: ctx,
catalog: catalog,
collections: nil,
segments: nil,
channelCPs: nil,
chunkManager: nil,
indexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: "_default_idx",
IsDeleted: false,
CreateTime: 10,
TypeParams: nil,
IndexParams: nil,
IsAutoIndex: false,
UserIndexParams: nil,
},
indexID + 1: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID + 1,
IndexID: indexID + 1,
IndexName: "_default_idx_101",
IsDeleted: true,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
IsAutoIndex: false,
UserIndexParams: nil,
},
},
collID + 1: {
indexID + 10: {
TenantID: "",
CollectionID: collID + 1,
FieldID: fieldID + 10,
IndexID: indexID + 10,
IndexName: "index",
IsDeleted: true,
CreateTime: 10,
TypeParams: nil,
IndexParams: nil,
IsAutoIndex: false,
UserIndexParams: nil,
},
},
},
buildID2SegmentIndex: nil,
}
}
func TestGarbageCollector_recycleUnusedIndexes(t *testing.T) {
t.Run("success", func(t *testing.T) {
gc := &garbageCollector{
meta: createMetaForRecycleUnusedIndexes(&datacoord.Catalog{Txn: &mockEtcdKv{}}),
}
gc.recycleUnusedIndexes()
})
t.Run("fail", func(t *testing.T) {
gc := &garbageCollector{
meta: createMetaForRecycleUnusedIndexes(&datacoord.Catalog{Txn: &removeFailKV{}}),
}
gc.recycleUnusedIndexes()
})
}
func createMetaForRecycleUnusedSegIndexes(catalog *datacoord.Catalog) *meta {
var (
ctx = context.Background()
collID = UniqueID(100)
partID = UniqueID(200)
//fieldID = UniqueID(300)
indexID = UniqueID(400)
segID = UniqueID(500)
)
return &meta{
RWMutex: sync.RWMutex{},
ctx: ctx,
catalog: catalog,
collections: nil,
segments: &SegmentsInfo{
segments: map[UniqueID]*SegmentInfo{
segID: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 1026,
State: commonpb.SegmentState_Flushed,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
},
},
segID + 1: {
SegmentInfo: nil,
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
},
},
},
},
channelCPs: nil,
chunkManager: nil,
indexes: map[UniqueID]map[UniqueID]*model.Index{},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
},
}
}
func TestGarbageCollector_recycleUnusedSegIndexes(t *testing.T) {
t.Run("success", func(t *testing.T) {
gc := &garbageCollector{
meta: createMetaForRecycleUnusedSegIndexes(&datacoord.Catalog{Txn: &mockEtcdKv{}}),
}
gc.recycleUnusedSegIndexes()
})
t.Run("fail", func(t *testing.T) {
gc := &garbageCollector{
meta: createMetaForRecycleUnusedSegIndexes(&datacoord.Catalog{Txn: &removeFailKV{}}),
}
gc.recycleUnusedSegIndexes()
})
}
func createMetaTableForRecycleUnusedIndexFiles(catalog *datacoord.Catalog) *meta {
var (
ctx = context.Background()
collID = UniqueID(100)
partID = UniqueID(200)
//fieldID = UniqueID(300)
indexID = UniqueID(400)
segID = UniqueID(500)
buildID = UniqueID(600)
)
return &meta{
RWMutex: sync.RWMutex{},
ctx: ctx,
catalog: catalog,
collections: nil,
segments: &SegmentsInfo{
segments: map[UniqueID]*SegmentInfo{
segID: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 1026,
State: commonpb.SegmentState_Flushed,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
},
},
segID + 1: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID + 1,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 1026,
State: commonpb.SegmentState_Flushed,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
},
},
},
indexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: "_default_idx",
IsDeleted: false,
CreateTime: 10,
TypeParams: nil,
IndexParams: nil,
IsAutoIndex: false,
UserIndexParams: nil,
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1026,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 1,
IndexVersion: 1,
IndexState: commonpb.IndexState_InProgress,
FailReason: "",
IsDeleted: false,
CreateTime: 10,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
}
}
func TestGarbageCollector_recycleUnusedIndexFiles(t *testing.T) {
t.Run("success", func(t *testing.T) {
cm := &mocks.ChunkManager{}
cm.EXPECT().RootPath().Return("root")
cm.EXPECT().ListWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return([]string{"a/b/c/", "a/b/600/", "a/b/601/", "a/b/602/"}, nil, nil)
cm.EXPECT().RemoveWithPrefix(mock.Anything, mock.Anything).Return(nil)
cm.EXPECT().Remove(mock.Anything, mock.Anything).Return(nil)
gc := &garbageCollector{
meta: createMetaTableForRecycleUnusedIndexFiles(&datacoord.Catalog{Txn: &mockEtcdKv{}}),
option: GcOption{
cli: cm,
},
}
gc.recycleUnusedIndexFiles()
})
t.Run("list fail", func(t *testing.T) {
cm := &mocks.ChunkManager{}
cm.EXPECT().RootPath().Return("root")
cm.EXPECT().ListWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return(nil, nil, errors.New("error"))
gc := &garbageCollector{
meta: createMetaTableForRecycleUnusedIndexFiles(&datacoord.Catalog{Txn: &mockEtcdKv{}}),
option: GcOption{
cli: cm,
},
}
gc.recycleUnusedIndexFiles()
})
t.Run("remove fail", func(t *testing.T) {
cm := &mocks.ChunkManager{}
cm.EXPECT().RootPath().Return("root")
cm.EXPECT().Remove(mock.Anything, mock.Anything).Return(errors.New("error"))
cm.EXPECT().ListWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return([]string{"a/b/c/", "a/b/600/", "a/b/601/", "a/b/602/"}, nil, nil)
cm.EXPECT().RemoveWithPrefix(mock.Anything, mock.Anything).Return(nil)
gc := &garbageCollector{
meta: createMetaTableForRecycleUnusedIndexFiles(&datacoord.Catalog{Txn: &mockEtcdKv{}}),
option: GcOption{
cli: cm,
},
}
gc.recycleUnusedIndexFiles()
})
t.Run("remove with prefix fail", func(t *testing.T) {
cm := &mocks.ChunkManager{}
cm.EXPECT().RootPath().Return("root")
cm.EXPECT().Remove(mock.Anything, mock.Anything).Return(errors.New("error"))
cm.EXPECT().ListWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return([]string{"a/b/c/", "a/b/600/", "a/b/601/", "a/b/602/"}, nil, nil)
cm.EXPECT().RemoveWithPrefix(mock.Anything, mock.Anything).Return(errors.New("error"))
gc := &garbageCollector{
meta: createMetaTableForRecycleUnusedIndexFiles(&datacoord.Catalog{Txn: &mockEtcdKv{}}),
option: GcOption{
cli: cm,
},
}
gc.recycleUnusedIndexFiles()
})
}
func TestGarbageCollector_clearETCD(t *testing.T) {
m := &meta{
catalog: &datacoord.Catalog{Txn: &mockEtcdKv{}},
segments: &SegmentsInfo{
map[UniqueID]*SegmentInfo{
segID: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 5000,
State: commonpb.SegmentState_Dropped,
MaxRowNum: 65536,
DroppedAt: 0,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 1024,
WriteHandoff: false,
},
},
},
segID + 1: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID + 1,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 5000,
State: commonpb.SegmentState_Dropped,
MaxRowNum: 65536,
DroppedAt: 0,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{
indexID: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file3", "file4"},
IndexSize: 1024,
WriteHandoff: false,
},
},
},
segID + 2: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID + 2,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 10000,
State: commonpb.SegmentState_Dropped,
MaxRowNum: 65536,
DroppedAt: 10,
CompactionFrom: []int64{segID, segID + 1},
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{},
},
segID + 3: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID + 3,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 2000,
State: commonpb.SegmentState_Dropped,
MaxRowNum: 65536,
DroppedAt: 10,
CompactionFrom: nil,
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{},
},
segID + 4: {
SegmentInfo: &datapb.SegmentInfo{
ID: segID + 4,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 12000,
State: commonpb.SegmentState_Flushed,
MaxRowNum: 65536,
DroppedAt: 10,
CompactionFrom: []int64{segID + 2, segID + 3},
},
segmentIndexes: map[UniqueID]*model.SegmentIndex{},
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 1024,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 5000,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 0,
IndexVersion: 1,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: []string{"file3", "file4"},
IndexSize: 1024,
WriteHandoff: false,
},
},
indexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: indexName,
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
IsAutoIndex: false,
UserIndexParams: nil,
},
},
},
collections: map[UniqueID]*collectionInfo{
collID: {
ID: collID,
Schema: &schemapb.CollectionSchema{
Name: "",
Description: "",
AutoID: false,
Fields: []*schemapb.FieldSchema{
{
FieldID: fieldID,
Name: "",
IsPrimaryKey: false,
Description: "",
DataType: schemapb.DataType_FloatVector,
TypeParams: nil,
IndexParams: nil,
AutoID: false,
State: 0,
},
},
},
Partitions: nil,
StartPositions: nil,
Properties: nil,
},
},
}
cm := &mocks.ChunkManager{}
cm.EXPECT().Remove(mock.Anything, mock.Anything).Return(nil)
gc := &garbageCollector{
option: GcOption{
cli: &mocks.ChunkManager{},
dropTolerance: 1,
},
meta: m,
handler: newMockHandlerWithMeta(m),
}
gc.clearEtcd()
segA := gc.meta.GetSegmentUnsafe(segID)
assert.NotNil(t, segA)
segB := gc.meta.GetSegmentUnsafe(segID + 1)
assert.NotNil(t, segB)
segC := gc.meta.GetSegmentUnsafe(segID + 2)
assert.NotNil(t, segC)
segD := gc.meta.GetSegmentUnsafe(segID + 3)
assert.NotNil(t, segD)
segE := gc.meta.GetSegmentUnsafe(segID + 4)
assert.NotNil(t, segE)
err := gc.meta.AddSegmentIndex(&model.SegmentIndex{
SegmentID: segID + 4,
CollectionID: collID,
PartitionID: partID,
NumRows: 12000,
IndexID: indexID,
BuildID: buildID + 4,
})
assert.NoError(t, err)
err = gc.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: buildID + 4,
State: commonpb.IndexState_Finished,
IndexFileKeys: []string{"file1", "file2", "file3", "file4"},
SerializedSize: 10240,
FailReason: "",
})
assert.NoError(t, err)
gc.clearEtcd()
//segA := gc.meta.GetSegmentUnsafe(segID)
//assert.NotNil(t, segA)
//segB := gc.meta.GetSegmentUnsafe(segID + 1)
//assert.NotNil(t, segB)
segC = gc.meta.GetSegmentUnsafe(segID + 2)
assert.Nil(t, segC)
segD = gc.meta.GetSegmentUnsafe(segID + 3)
assert.Nil(t, segD)
segE = gc.meta.GetSegmentUnsafe(segID + 4)
assert.NotNil(t, segE)
gc.clearEtcd()
segA = gc.meta.GetSegmentUnsafe(segID)
assert.Nil(t, segA)
segB = gc.meta.GetSegmentUnsafe(segID + 1)
assert.Nil(t, segB)
}

View File

@ -104,7 +104,7 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni
return s.InsertChannel == channel.Name && !s.GetIsFake()
})
segmentInfos := make(map[int64]*SegmentInfo)
indexedSegments := FilterInIndexedSegments(h, h.s.indexCoord, segments...)
indexedSegments := FilterInIndexedSegments(h, h.s.meta, segments...)
indexed := make(typeutil.UniqueSet)
for _, segment := range indexedSegments {
indexed.Insert(segment.GetID())
@ -113,6 +113,7 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni
zap.Int64("collectionID", channel.CollectionID),
zap.String("channel", channel.Name),
zap.Int("numOfSegments", len(segments)),
zap.Int("indexed segment", len(indexedSegments)),
)
var (
indexedIDs = make(typeutil.UniqueSet)
@ -148,6 +149,68 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni
}
}
//var (
// indexedIDs = make(typeutil.UniqueSet)
// unIndexedIDs = make(typeutil.UniqueSet)
// growingIDs = make(typeutil.UniqueSet)
// droppedIDs = make(typeutil.UniqueSet)
//)
//for _, s := range segments {
// if (partitionID > allPartitionID && s.PartitionID != partitionID) ||
// (s.GetStartPosition() == nil && s.GetDmlPosition() == nil) {
// continue
// }
// if s.GetIsImporting() {
// // Skip bulk insert segments.
// continue
// }
// segmentInfos[s.GetID()] = s
// if s.GetState() == commonpb.SegmentState_Dropped {
// droppedIDs.Insert(s.GetID())
// } else if s.GetState() == commonpb.SegmentState_Growing {
// growingIDs.Insert(s.GetID())
// } else if indexed.Contain(s.GetID()) {
// indexedIDs.Insert(s.GetID())
// } else {
// unIndexedIDs.Insert(s.GetID())
// }
//}
//hasUnIndexed := true
//for hasUnIndexed {
// hasUnIndexed = false
// for id := range unIndexedIDs {
// // Indexed segments are compacted to a raw segment,
// // replace it with the indexed ones
// if indexed.Contain(id) {
// unIndexedIDs.Remove(id)
// indexedIDs.Insert(id)
// continue
// }
// if len(segmentInfos[id].GetCompactionFrom()) > 0 {
// unIndexedIDs.Remove(id)
// for _, segID := range segmentInfos[id].GetCompactionFrom() {
// if indexed.Contain(segID) {
// indexedIDs.Insert(segID)
// } else {
// unIndexedIDs.Insert(id)
// hasUnIndexed = true
// }
// }
// droppedIDs.Remove(segmentInfos[id].GetCompactionFrom()...)
// }
// }
//}
//
//return &datapb.VchannelInfo{
// CollectionID: channel.CollectionID,
// ChannelName: channel.Name,
// SeekPosition: h.GetChannelSeekPosition(channel, partitionID),
// IndexedSegmentIds: indexed.Collect(),
// FlushedSegmentIds: unIndexedIDs.Collect(),
// UnflushedSegmentIds: growingIDs.Collect(),
// DroppedSegmentIds: droppedIDs.Collect(),
//}
return &datapb.VchannelInfo{
CollectionID: channel.CollectionID,
ChannelName: channel.Name,
@ -155,6 +218,7 @@ func (h *ServerHandler) GetQueryVChanPositions(channel *channel, partitionID Uni
FlushedSegmentIds: indexedIDs.Collect(),
UnflushedSegmentIds: unIndexedIDs.Collect(),
DroppedSegmentIds: droppedIDs.Collect(),
// IndexedSegmentIds: indexed.Collect(),
}
}

View File

@ -0,0 +1,449 @@
// 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 datacoord
import (
"context"
"errors"
"path"
"sync"
"time"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
)
type indexTaskState int32
const (
// when we receive a index task
indexTaskInit indexTaskState = iota
// we've sent index task to scheduler, and wait for building index.
indexTaskInProgress
// task done, wait to be cleaned
indexTaskDone
// index task need to retry.
indexTaskRetry
// task has been deleted.
indexTaskDeleted
// task needs to prepare segment info on IndexNode
indexTaskPrepare
reqTimeoutInterval = time.Second * 10
)
var TaskStateNames = map[indexTaskState]string{
0: "Init",
1: "InProgress",
2: "Done",
3: "Retry",
4: "Deleted",
5: "Prepare",
}
func (x indexTaskState) String() string {
ret, ok := TaskStateNames[x]
if !ok {
return "None"
}
return ret
}
type indexBuilder struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
taskMutex sync.RWMutex
scheduleDuration time.Duration
// TODO @xiaocai2333: use priority queue
tasks map[int64]indexTaskState
notifyChan chan struct{}
meta *meta
policy buildIndexPolicy
nodeManager *IndexNodeManager
chunkManager storage.ChunkManager
}
func newIndexBuilder(ctx context.Context, metaTable *meta, nodeManager *IndexNodeManager, chunkManager storage.ChunkManager) *indexBuilder {
ctx, cancel := context.WithCancel(ctx)
ib := &indexBuilder{
ctx: ctx,
cancel: cancel,
meta: metaTable,
tasks: make(map[int64]indexTaskState),
notifyChan: make(chan struct{}, 1),
scheduleDuration: time.Second,
policy: defaultBuildIndexPolicy,
nodeManager: nodeManager,
chunkManager: chunkManager,
}
ib.reloadFromKV()
return ib
}
func (ib *indexBuilder) Start() {
ib.wg.Add(1)
go ib.schedule()
}
func (ib *indexBuilder) Stop() {
ib.cancel()
ib.wg.Wait()
}
func (ib *indexBuilder) reloadFromKV() {
segments := ib.meta.GetAllSegmentsUnsafe()
for _, segment := range segments {
for _, segIndex := range segment.segmentIndexes {
if segIndex.IndexState == commonpb.IndexState_Unissued {
ib.tasks[segIndex.BuildID] = indexTaskInit
} else if segIndex.IndexState == commonpb.IndexState_InProgress {
ib.tasks[segIndex.BuildID] = indexTaskInProgress
}
}
}
}
// notify is an unblocked notify function
func (ib *indexBuilder) notify() {
select {
case ib.notifyChan <- struct{}{}:
default:
}
}
func (ib *indexBuilder) enqueue(buildID UniqueID) {
defer ib.notify()
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
if _, ok := ib.tasks[buildID]; !ok {
ib.tasks[buildID] = indexTaskInit
}
log.Info("indexBuilder enqueue task", zap.Int64("buildID", buildID))
}
func (ib *indexBuilder) schedule() {
// receive notifyChan
// time ticker
log.Ctx(ib.ctx).Info("index builder schedule loop start")
defer ib.wg.Done()
ticker := time.NewTicker(ib.scheduleDuration)
defer ticker.Stop()
for {
select {
case <-ib.ctx.Done():
log.Ctx(ib.ctx).Warn("index builder ctx done")
return
case _, ok := <-ib.notifyChan:
if ok {
ib.run()
}
// !ok means indexBuild is closed.
case <-ticker.C:
ib.run()
}
}
}
func (ib *indexBuilder) run() {
ib.taskMutex.RLock()
buildIDs := make([]UniqueID, 0, len(ib.tasks))
for tID := range ib.tasks {
buildIDs = append(buildIDs, tID)
}
ib.taskMutex.RUnlock()
if len(buildIDs) > 0 {
log.Ctx(ib.ctx).Info("index builder task schedule", zap.Int("task num", len(buildIDs)))
}
ib.policy(buildIDs)
for _, buildID := range buildIDs {
ok := ib.process(buildID)
if !ok {
log.Ctx(ib.ctx).Info("there is no IndexNode available or etcd is not serviceable, wait a minute...")
break
}
}
}
func (ib *indexBuilder) process(buildID UniqueID) bool {
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)
}
meta, exist := ib.meta.GetIndexJob(buildID)
if !exist {
log.Ctx(ib.ctx).Debug("index task has not exist in meta table, remove task", zap.Int64("buildID", buildID))
deleteFunc(buildID)
return true
}
switch state {
case indexTaskInit:
segment := ib.meta.GetSegmentUnsafe(meta.SegmentID)
if !isSegmentHealthy(segment) || !ib.meta.IsIndexExist(meta.CollectionID, meta.IndexID) {
log.Ctx(ib.ctx).Info("task is no need to build index, remove it", zap.Int64("buildID", buildID))
deleteFunc(buildID)
return true
}
indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID)
if isFlatIndex(getIndexType(indexParams)) || meta.NumRows < Params.DataCoordCfg.MinSegmentNumRowsToEnableIndex.GetAsInt64() {
log.Ctx(ib.ctx).Debug("segment does not need index really", zap.Int64("buildID", buildID),
zap.Int64("segID", meta.SegmentID), zap.Int64("num rows", meta.NumRows))
if err := ib.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: buildID,
State: commonpb.IndexState_Finished,
IndexFileKeys: nil,
SerializedSize: 0,
FailReason: "",
}); err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord update index state fail", zap.Int64("buildID", buildID), zap.Error(err))
return false
}
updateStateFunc(buildID, indexTaskDone)
return true
}
// peek client
// if all IndexNodes are executing task, wait for one of them to finish the task.
nodeID, client := ib.nodeManager.PeekClient(meta)
if client == nil {
log.Ctx(ib.ctx).RatedInfo(5, "index builder peek client error, there is no available")
return false
}
// update version and set nodeID
if err := ib.meta.UpdateVersion(buildID, nodeID); err != nil {
log.Ctx(ib.ctx).Warn("index builder update index version failed", zap.Int64("build", buildID), zap.Error(err))
return false
}
binLogs := make([]string, 0)
fieldID := ib.meta.GetFieldIDByIndexID(meta.CollectionID, meta.IndexID)
for _, fieldBinLog := range segment.GetBinlogs() {
if fieldBinLog.GetFieldID() == fieldID {
for _, binLog := range fieldBinLog.GetBinlogs() {
binLogs = append(binLogs, binLog.LogPath)
}
break
}
}
typeParams := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID)
var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType.GetValue() == "local" {
storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path.GetValue(),
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
} else {
storageConfig = &indexpb.StorageConfig{
Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath.GetValue(),
UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
}
req := &indexpb.CreateJobRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
IndexFilePrefix: path.Join(ib.chunkManager.RootPath(), common.SegmentIndexPath),
BuildID: buildID,
DataPaths: binLogs,
IndexVersion: meta.IndexVersion + 1,
StorageConfig: storageConfig,
IndexParams: indexParams,
TypeParams: typeParams,
NumRows: meta.NumRows,
}
if err := ib.assignTask(client, req); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder assign task to IndexNode failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
return false
}
log.Ctx(ib.ctx).Info("index task assigned successfully", zap.Int64("buildID", buildID),
zap.Int64("segID", meta.SegmentID), zap.Int64("nodeID", nodeID))
// update index meta state to InProgress
if err := ib.meta.BuildIndex(buildID); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder update index meta to InProgress failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
return false
}
updateStateFunc(buildID, indexTaskInProgress)
case indexTaskDone:
if !ib.dropIndexTask(buildID, meta.NodeID) {
return true
}
deleteFunc(buildID)
case indexTaskRetry:
if !ib.dropIndexTask(buildID, meta.NodeID) {
return true
}
updateStateFunc(buildID, indexTaskInit)
case indexTaskDeleted:
deleteFunc(buildID)
default:
// state: in_progress
updateStateFunc(buildID, ib.getTaskState(buildID, meta.NodeID))
}
return true
}
func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
client, exist := ib.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel()
response, err := client.QueryJobs(ctx1, &indexpb.QueryJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []int64{buildID},
})
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord get jobs info from IndexNode fail", zap.Int64("nodeID", nodeID),
zap.Error(err))
return indexTaskInProgress
}
if response.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Ctx(ib.ctx).Warn("IndexCoord get jobs info from IndexNode fail", zap.Int64("nodeID", nodeID),
zap.Int64("buildID", buildID), zap.String("fail reason", response.Status.Reason))
return indexTaskInProgress
}
// indexInfos length is always one.
for _, info := range response.IndexInfos {
if info.BuildID == buildID {
if info.State == commonpb.IndexState_Failed || info.State == commonpb.IndexState_Finished {
log.Ctx(ib.ctx).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.Ctx(ib.ctx).Warn("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.Ctx(ib.ctx).Info("this task should be retry", zap.Int64("buildID", buildID), zap.String("fail reason", info.FailReason))
return indexTaskRetry
}
return indexTaskInProgress
}
}
log.Ctx(ib.ctx).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
log.Ctx(ib.ctx).Info("this task should be retry, indexNode is no longer exist", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return indexTaskRetry
}
func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
client, exist := ib.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel()
status, err := client.DropJobs(ctx1, &indexpb.DropJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []UniqueID{buildID},
})
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord notify IndexNode drop the index task fail", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
return false
}
if status.ErrorCode != commonpb.ErrorCode_Success {
log.Ctx(ib.ctx).Warn("IndexCoord notify IndexNode drop the index task fail", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.String("fail reason", status.Reason))
return false
}
log.Ctx(ib.ctx).Info("IndexCoord notify IndexNode drop the index task success",
zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
return true
}
log.Ctx(ib.ctx).Info("IndexNode no longer exist, no need to drop index task",
zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
return true
}
// assignTask sends the index task to the IndexNode, it has a timeout interval, if the IndexNode doesn't respond within
// the interval, it is considered that the task sending failed.
func (ib *indexBuilder) assignTask(builderClient types.IndexNode, req *indexpb.CreateJobRequest) error {
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
defer cancel()
resp, err := builderClient.CreateJob(ctx, req)
if err != nil {
log.Error("IndexCoord assignmentTasksLoop builderClient.CreateIndex failed", zap.Error(err))
return err
}
if resp.ErrorCode != commonpb.ErrorCode_Success {
log.Error("IndexCoord assignmentTasksLoop builderClient.CreateIndex failed", zap.String("Reason", resp.Reason))
return errors.New(resp.Reason)
}
return nil
}
func (ib *indexBuilder) nodeDown(nodeID UniqueID) {
defer ib.notify()
metas := ib.meta.GetMetasByNodeID(nodeID)
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
for _, meta := range metas {
if ib.tasks[meta.BuildID] != indexTaskDone {
ib.tasks[meta.BuildID] = indexTaskRetry
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,689 @@
// 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 datacoord contains core functions in datacoord
package datacoord
import (
"fmt"
"strconv"
"github.com/golang/protobuf/proto"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"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/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/prometheus/client_golang/prometheus"
)
func (m *meta) updateCollectionIndex(index *model.Index) {
if _, ok := m.indexes[index.CollectionID]; !ok {
m.indexes[index.CollectionID] = make(map[UniqueID]*model.Index)
}
m.indexes[index.CollectionID][index.IndexID] = index
}
func (m *meta) updateSegmentIndex(segIdx *model.SegmentIndex) {
m.segments.SetSegmentIndex(segIdx.SegmentID, segIdx)
m.buildID2SegmentIndex[segIdx.BuildID] = segIdx
}
func (m *meta) alterSegmentIndexes(segIdxes []*model.SegmentIndex) error {
err := m.catalog.AlterSegmentIndexes(m.ctx, segIdxes)
if err != nil {
log.Error("failed to alter segments index in meta store", zap.Int("segment indexes num", len(segIdxes)),
zap.Error(err))
return err
}
for _, segIdx := range segIdxes {
m.updateSegmentIndex(segIdx)
}
return nil
}
func (m *meta) updateIndexMeta(index *model.Index, updateFunc func(clonedIndex *model.Index) error) error {
return updateFunc(model.CloneIndex(index))
}
func (m *meta) updateSegIndexMeta(segIdx *model.SegmentIndex, updateFunc func(clonedSegIdx *model.SegmentIndex) error) error {
return updateFunc(model.CloneSegmentIndex(segIdx))
}
func (m *meta) updateIndexTasksMetrics() {
taskMetrics := make(map[UniqueID]map[commonpb.IndexState]int)
for _, segIdx := range m.buildID2SegmentIndex {
if segIdx.IsDeleted {
continue
}
if _, ok := taskMetrics[segIdx.CollectionID]; !ok {
taskMetrics[segIdx.CollectionID] = make(map[commonpb.IndexState]int)
taskMetrics[segIdx.CollectionID][commonpb.IndexState_Unissued] = 0
taskMetrics[segIdx.CollectionID][commonpb.IndexState_InProgress] = 0
taskMetrics[segIdx.CollectionID][commonpb.IndexState_Finished] = 0
taskMetrics[segIdx.CollectionID][commonpb.IndexState_Failed] = 0
}
taskMetrics[segIdx.CollectionID][segIdx.IndexState]++
}
for collID, m := range taskMetrics {
for k, v := range m {
switch k {
case commonpb.IndexState_Unissued:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.UnissuedIndexTaskLabel).Set(float64(v))
case commonpb.IndexState_InProgress:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.InProgressIndexTaskLabel).Set(float64(v))
case commonpb.IndexState_Finished:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.FinishedIndexTaskLabel).Set(float64(v))
case commonpb.IndexState_Failed:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.FailedIndexTaskLabel).Set(float64(v))
}
}
}
}
func checkParams(fieldIndex *model.Index, req *datapb.CreateIndexRequest) bool {
if len(fieldIndex.TypeParams) != len(req.TypeParams) {
return false
}
notEq := false
for _, param1 := range fieldIndex.TypeParams {
exist := false
for _, param2 := range req.TypeParams {
if param2.Key == param1.Key && param2.Value == param1.Value {
exist = true
}
}
if !exist {
notEq = true
break
}
}
if notEq {
return false
}
if len(fieldIndex.IndexParams) != len(req.IndexParams) {
return false
}
for _, param1 := range fieldIndex.IndexParams {
exist := false
for _, param2 := range req.IndexParams {
if param2.Key == param1.Key && param2.Value == param1.Value {
exist = true
}
}
if !exist {
notEq = true
break
}
}
return !notEq
}
func (m *meta) CanCreateIndex(req *datapb.CreateIndexRequest) (UniqueID, error) {
m.RLock()
defer m.RUnlock()
indexes, ok := m.indexes[req.CollectionID]
if !ok {
return 0, nil
}
for _, index := range indexes {
if index.IsDeleted {
continue
}
if req.IndexName == index.IndexName {
if req.FieldID == index.FieldID && checkParams(index, req) {
return index.IndexID, nil
}
errMsg := "at most one distinct index is allowed per field"
log.Warn(errMsg,
zap.String("source index", fmt.Sprintf("{index_name: %s, field_id: %d, index_params: %v, type_params: %v}", index.IndexName, index.FieldID, index.IndexParams, index.TypeParams)),
zap.String("current index", fmt.Sprintf("{index_name: %s, field_id: %d, index_params: %v, type_params: %v}", req.GetIndexName(), req.GetFieldID(), req.GetIndexParams(), req.GetTypeParams())))
return 0, fmt.Errorf("CreateIndex failed: %s", errMsg)
}
if req.FieldID == index.FieldID {
// creating multiple indexes on same field is not supported
errMsg := "CreateIndex failed: creating multiple indexes on same field is not supported"
log.Warn(errMsg)
return 0, fmt.Errorf(errMsg)
}
}
return 0, nil
}
// HasSameReq determine whether there are same indexing tasks.
func (m *meta) HasSameReq(req *datapb.CreateIndexRequest) (bool, UniqueID) {
m.RLock()
defer m.RUnlock()
for _, fieldIndex := range m.indexes[req.CollectionID] {
if fieldIndex.IsDeleted {
continue
}
if fieldIndex.FieldID != req.FieldID || fieldIndex.IndexName != req.IndexName {
continue
}
if !checkParams(fieldIndex, req) {
continue
}
log.Debug("has same index", zap.Int64("collectionID", req.CollectionID),
zap.Int64("fieldID", req.FieldID), zap.String("indexName", req.IndexName),
zap.Int64("indexID", fieldIndex.IndexID))
return true, fieldIndex.IndexID
}
return false, 0
}
func (m *meta) CreateIndex(index *model.Index) error {
log.Info("meta update: CreateIndex", zap.Int64("collectionID", index.CollectionID),
zap.Int64("fieldID", index.FieldID), zap.Int64("indexID", index.IndexID), zap.String("indexName", index.IndexName))
m.Lock()
defer m.Unlock()
if err := m.catalog.CreateIndex(m.ctx, index); err != nil {
log.Error("meta update: CreateIndex save meta fail", zap.Int64("collectionID", index.CollectionID),
zap.Int64("fieldID", index.FieldID), zap.Int64("indexID", index.IndexID),
zap.String("indexName", index.IndexName), zap.Error(err))
return err
}
m.updateCollectionIndex(index)
log.Info("meta update: CreateIndex success", zap.Int64("collectionID", index.CollectionID),
zap.Int64("fieldID", index.FieldID), zap.Int64("indexID", index.IndexID), zap.String("indexName", index.IndexName))
return nil
}
// AddSegmentIndex adds the index meta corresponding the indexBuildID to meta table.
func (m *meta) AddSegmentIndex(segIndex *model.SegmentIndex) error {
m.Lock()
defer m.Unlock()
buildID := segIndex.BuildID
log.Info("meta update: adding segment index", zap.Int64("collID", segIndex.CollectionID),
zap.Int64("segID", segIndex.SegmentID), zap.Int64("indexID", segIndex.IndexID),
zap.Int64("buildID", buildID))
segIndex.IndexState = commonpb.IndexState_Unissued
if err := m.catalog.CreateSegmentIndex(m.ctx, segIndex); err != nil {
log.Warn("meta update: adding segment index failed",
zap.Int64("segment ID", segIndex.SegmentID), zap.Int64("indexID", segIndex.IndexID),
zap.Int64("buildID", segIndex.BuildID), zap.Error(err))
return err
}
m.updateSegmentIndex(segIndex)
log.Info("meta update: adding segment index success", zap.Int64("collID", segIndex.CollectionID),
zap.Int64("segID", segIndex.SegmentID), zap.Int64("indexID", segIndex.IndexID),
zap.Int64("buildID", buildID))
m.updateIndexTasksMetrics()
return nil
}
func (m *meta) GetIndexIDByName(collID int64, indexName string) map[int64]uint64 {
m.RLock()
defer m.RUnlock()
indexID2CreateTs := make(map[int64]uint64)
fieldIndexes, ok := m.indexes[collID]
if !ok {
return indexID2CreateTs
}
for _, index := range fieldIndexes {
if !index.IsDeleted && (indexName == "" || index.IndexName == indexName) {
indexID2CreateTs[index.IndexID] = index.CreateTime
}
}
return indexID2CreateTs
}
type IndexState struct {
state commonpb.IndexState
failReason string
}
func (m *meta) GetSegmentIndexState(collID, segmentID UniqueID) IndexState {
m.RLock()
defer m.RUnlock()
state := IndexState{
state: commonpb.IndexState_IndexStateNone,
failReason: "",
}
fieldIndexes, ok := m.indexes[collID]
if !ok {
state.failReason = fmt.Sprintf("collection not exist with ID: %d", collID)
return state
}
segment := m.segments.GetSegment(segmentID)
if segment != nil {
for indexID, index := range fieldIndexes {
if !index.IsDeleted {
if segIdx, ok := segment.segmentIndexes[indexID]; ok {
if segIdx.IndexState != commonpb.IndexState_Finished {
state.state = segIdx.IndexState
state.failReason = segIdx.FailReason
break
}
state.state = commonpb.IndexState_Finished
continue
}
state.state = commonpb.IndexState_Unissued
break
}
}
return state
}
state.failReason = fmt.Sprintf("segment is not exist with ID: %d", segmentID)
return state
}
func (m *meta) GetSegmentIndexStateOnField(collID, segmentID, fieldID UniqueID) IndexState {
m.RLock()
defer m.RUnlock()
state := IndexState{
state: commonpb.IndexState_IndexStateNone,
failReason: "",
}
fieldIndexes, ok := m.indexes[collID]
if !ok {
state.failReason = fmt.Sprintf("collection not exist with ID: %d", collID)
return state
}
segment := m.segments.GetSegment(segmentID)
if segment != nil {
for indexID, index := range fieldIndexes {
if index.FieldID == fieldID && !index.IsDeleted {
if segIdx, ok := segment.segmentIndexes[indexID]; ok {
state.state = segIdx.IndexState
state.failReason = segIdx.FailReason
return state
}
state.state = commonpb.IndexState_Unissued
return state
}
}
state.failReason = fmt.Sprintf("there is no index on fieldID: %d", fieldID)
return state
}
state.failReason = fmt.Sprintf("segment is not exist with ID: %d", segmentID)
return state
}
// GetIndexesForCollection gets all indexes info with the specified collection.
func (m *meta) GetIndexesForCollection(collID UniqueID, indexName string) []*model.Index {
m.RLock()
defer m.RUnlock()
indexInfos := make([]*model.Index, 0)
for _, index := range m.indexes[collID] {
if index.IsDeleted {
continue
}
if indexName == "" || indexName == index.IndexName {
indexInfos = append(indexInfos, model.CloneIndex(index))
}
}
return indexInfos
}
// MarkIndexAsDeleted will mark the corresponding index as deleted, and recycleUnusedIndexFiles will recycle these tasks.
func (m *meta) MarkIndexAsDeleted(collID UniqueID, indexIDs []UniqueID) error {
log.Info("IndexCoord metaTable MarkIndexAsDeleted", zap.Int64("collID", collID),
zap.Int64s("indexIDs", indexIDs))
m.Lock()
defer m.Unlock()
fieldIndexes, ok := m.indexes[collID]
if !ok {
return nil
}
indexes := make([]*model.Index, 0)
for _, indexID := range indexIDs {
index, ok := fieldIndexes[indexID]
if !ok || index.IsDeleted {
continue
}
clonedIndex := model.CloneIndex(index)
clonedIndex.IsDeleted = true
indexes = append(indexes, clonedIndex)
}
if len(indexes) == 0 {
return nil
}
err := m.catalog.AlterIndexes(m.ctx, indexes)
if err != nil {
log.Error("failed to alter index meta in meta store", zap.Int("indexes num", len(indexes)), zap.Error(err))
return err
}
for _, index := range indexes {
m.indexes[index.CollectionID][index.IndexID] = index
}
log.Info("IndexCoord metaTable MarkIndexAsDeleted success", zap.Int64("collID", collID), zap.Int64s("indexIDs", indexIDs))
return nil
}
func (m *meta) GetSegmentIndexes(segID UniqueID) []*model.SegmentIndex {
m.RLock()
defer m.RUnlock()
segIndexInfos := make([]*model.SegmentIndex, 0)
segment := m.segments.GetSegment(segID)
if segment == nil {
return segIndexInfos
}
fieldIndex, ok := m.indexes[segment.CollectionID]
if !ok {
return segIndexInfos
}
for _, segIdx := range segment.segmentIndexes {
if index, ok := fieldIndex[segIdx.IndexID]; ok && !index.IsDeleted {
segIndexInfos = append(segIndexInfos, model.CloneSegmentIndex(segIdx))
}
}
return segIndexInfos
}
func (m *meta) GetFieldIDByIndexID(collID, indexID UniqueID) UniqueID {
m.RLock()
defer m.RUnlock()
if fieldIndexes, ok := m.indexes[collID]; ok {
if index, ok := fieldIndexes[indexID]; ok {
return index.FieldID
}
}
return 0
}
func (m *meta) GetIndexNameByID(collID, indexID UniqueID) string {
m.RLock()
defer m.RUnlock()
if fieldIndexes, ok := m.indexes[collID]; ok {
if index, ok := fieldIndexes[indexID]; ok {
return index.IndexName
}
}
return ""
}
func (m *meta) GetIndexParams(collID, indexID UniqueID) []*commonpb.KeyValuePair {
m.RLock()
defer m.RUnlock()
fieldIndexes, ok := m.indexes[collID]
if !ok {
return nil
}
index, ok := fieldIndexes[indexID]
if !ok {
return nil
}
indexParams := make([]*commonpb.KeyValuePair, 0, len(index.IndexParams))
for _, param := range index.IndexParams {
indexParams = append(indexParams, proto.Clone(param).(*commonpb.KeyValuePair))
}
return indexParams
}
func (m *meta) GetTypeParams(collID, indexID UniqueID) []*commonpb.KeyValuePair {
m.RLock()
defer m.RUnlock()
fieldIndexes, ok := m.indexes[collID]
if !ok {
return nil
}
index, ok := fieldIndexes[indexID]
if !ok {
return nil
}
typeParams := make([]*commonpb.KeyValuePair, 0, len(index.TypeParams))
for _, param := range index.TypeParams {
typeParams = append(typeParams, proto.Clone(param).(*commonpb.KeyValuePair))
}
return typeParams
}
func (m *meta) GetIndexJob(buildID UniqueID) (*model.SegmentIndex, bool) {
m.RLock()
defer m.RUnlock()
segIdx, ok := m.buildID2SegmentIndex[buildID]
if ok {
return model.CloneSegmentIndex(segIdx), true
}
return nil, false
}
func (m *meta) IsIndexExist(collID, indexID UniqueID) bool {
m.RLock()
defer m.RUnlock()
fieldIndexes, ok := m.indexes[collID]
if !ok {
return false
}
if index, ok := fieldIndexes[indexID]; !ok || index.IsDeleted {
return false
}
return true
}
// UpdateVersion updates the version and nodeID of the index meta, whenever the task is built once, the version will be updated once.
func (m *meta) UpdateVersion(buildID UniqueID, nodeID UniqueID) error {
m.Lock()
defer m.Unlock()
log.Debug("IndexCoord metaTable UpdateVersion receive", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
segIdx, ok := m.buildID2SegmentIndex[buildID]
if !ok {
return fmt.Errorf("there is no index with buildID: %d", buildID)
}
updateFunc := func(segIdx *model.SegmentIndex) error {
segIdx.NodeID = nodeID
segIdx.IndexVersion++
return m.alterSegmentIndexes([]*model.SegmentIndex{segIdx})
}
return m.updateSegIndexMeta(segIdx, updateFunc)
}
func (m *meta) FinishTask(taskInfo *indexpb.IndexTaskInfo) error {
m.Lock()
defer m.Unlock()
segIdx, ok := m.buildID2SegmentIndex[taskInfo.BuildID]
if !ok {
log.Warn("there is no index with buildID", zap.Int64("buildID", taskInfo.BuildID))
return nil
}
updateFunc := func(segIdx *model.SegmentIndex) error {
segIdx.IndexState = taskInfo.State
segIdx.IndexFileKeys = common.CloneStringList(taskInfo.IndexFileKeys)
segIdx.FailReason = taskInfo.FailReason
segIdx.IndexSize = taskInfo.SerializedSize
return m.alterSegmentIndexes([]*model.SegmentIndex{segIdx})
}
if err := m.updateSegIndexMeta(segIdx, updateFunc); err != nil {
return err
}
log.Info("finish index task success", zap.Int64("buildID", taskInfo.BuildID),
zap.String("state", taskInfo.GetState().String()), zap.String("fail reason", taskInfo.GetFailReason()))
m.updateIndexTasksMetrics()
return nil
}
// BuildIndex set the index state to be InProgress. It means IndexNode is building the index.
func (m *meta) BuildIndex(buildID UniqueID) error {
m.Lock()
defer m.Unlock()
segIdx, ok := m.buildID2SegmentIndex[buildID]
if !ok {
return fmt.Errorf("there is no index with buildID: %d", buildID)
}
updateFunc := func(segIdx *model.SegmentIndex) error {
segIdx.IndexState = commonpb.IndexState_InProgress
err := m.alterSegmentIndexes([]*model.SegmentIndex{segIdx})
if err != nil {
log.Error("meta Update: segment index in progress fail", zap.Int64("buildID", segIdx.BuildID), zap.Error(err))
return err
}
return nil
}
if err := m.updateSegIndexMeta(segIdx, updateFunc); err != nil {
return err
}
log.Info("meta update: segment index in progress success", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("segID", segIdx.SegmentID))
m.updateIndexTasksMetrics()
return nil
}
func (m *meta) GetAllSegIndexes() map[int64]*model.SegmentIndex {
m.RLock()
defer m.RUnlock()
segIndexes := make(map[int64]*model.SegmentIndex, len(m.buildID2SegmentIndex))
for buildID, segIndex := range m.buildID2SegmentIndex {
segIndexes[buildID] = model.CloneSegmentIndex(segIndex)
}
return segIndexes
}
func (m *meta) RemoveSegmentIndex(collID, partID, segID, indexID, buildID UniqueID) error {
m.Lock()
defer m.Unlock()
err := m.catalog.DropSegmentIndex(m.ctx, collID, partID, segID, buildID)
if err != nil {
return err
}
m.segments.DropSegmentIndex(segID, indexID)
delete(m.buildID2SegmentIndex, buildID)
m.updateIndexTasksMetrics()
return nil
}
func (m *meta) GetDeletedIndexes() []*model.Index {
m.RLock()
defer m.RUnlock()
deletedIndexes := make([]*model.Index, 0)
for _, fieldIndexes := range m.indexes {
for _, index := range fieldIndexes {
if index.IsDeleted {
deletedIndexes = append(deletedIndexes, index)
}
}
}
return deletedIndexes
}
func (m *meta) RemoveIndex(collID, indexID UniqueID) error {
m.Lock()
defer m.Unlock()
log.Info("IndexCoord meta table remove index", zap.Int64("collID", collID), zap.Int64("indexID", indexID))
err := m.catalog.DropIndex(m.ctx, collID, indexID)
if err != nil {
log.Info("IndexCoord meta table remove index fail", zap.Int64("collID", collID),
zap.Int64("indexID", indexID), zap.Error(err))
return err
}
delete(m.indexes[collID], indexID)
if len(m.indexes[collID]) == 0 {
delete(m.indexes, collID)
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.UnissuedIndexTaskLabel})
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.InProgressIndexTaskLabel})
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FinishedIndexTaskLabel})
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FailedIndexTaskLabel})
}
log.Info("IndexCoord meta table remove index success", zap.Int64("collID", collID), zap.Int64("indexID", indexID))
return nil
}
func (m *meta) CleanSegmentIndex(buildID UniqueID) (bool, *model.SegmentIndex) {
m.RLock()
defer m.RUnlock()
if segIndex, ok := m.buildID2SegmentIndex[buildID]; ok {
if segIndex.IndexState == commonpb.IndexState_Finished {
return true, model.CloneSegmentIndex(segIndex)
}
return false, model.CloneSegmentIndex(segIndex)
}
return true, nil
}
func (m *meta) GetHasUnindexTaskSegments() []*SegmentInfo {
m.RLock()
defer m.RUnlock()
segments := m.segments.GetSegments()
var ret []*SegmentInfo
for _, segment := range segments {
if !isFlush(segment) {
continue
}
if fieldIndexes, ok := m.indexes[segment.CollectionID]; ok {
for _, index := range fieldIndexes {
if _, ok := segment.segmentIndexes[index.IndexID]; !index.IsDeleted && !ok {
ret = append(ret, segment)
}
}
}
}
return ret
}
func (m *meta) GetMetasByNodeID(nodeID UniqueID) []*model.SegmentIndex {
m.RLock()
defer m.RUnlock()
metas := make([]*model.SegmentIndex, 0)
for _, segIndex := range m.buildID2SegmentIndex {
if segIndex.IsDeleted {
continue
}
if nodeID == segIndex.NodeID {
metas = append(metas, model.CloneSegmentIndex(segIndex))
}
}
return metas
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,599 @@
// 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 datacoord
import (
"context"
"fmt"
"time"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"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/util/metautil"
"github.com/milvus-io/milvus/internal/util/paramtable"
)
func (s *Server) startIndexService(ctx context.Context) {
s.indexBuilder.Start()
s.serverLoopWg.Add(1)
go s.createIndexForSegmentLoop(ctx)
}
func (s *Server) createIndexForSegment(segment *SegmentInfo, indexID UniqueID) error {
log.Info("create index for segment", zap.Int64("segID", segment.ID), zap.Int64("indexID", indexID))
buildID, err := s.allocator.allocID(context.Background())
if err != nil {
return err
}
segIndex := &model.SegmentIndex{
SegmentID: segment.ID,
CollectionID: segment.CollectionID,
PartitionID: segment.PartitionID,
NumRows: segment.NumOfRows,
IndexID: indexID,
BuildID: buildID,
CreateTime: segment.LastExpireTime,
WriteHandoff: false,
}
if err = s.meta.AddSegmentIndex(segIndex); err != nil {
return err
}
s.indexBuilder.enqueue(buildID)
return nil
}
func (s *Server) createIndexesForSegment(segment *SegmentInfo) error {
indexes := s.meta.GetIndexesForCollection(segment.CollectionID, "")
for _, index := range indexes {
if _, ok := segment.segmentIndexes[index.IndexID]; !ok {
if err := s.createIndexForSegment(segment, index.IndexID); err != nil {
log.Warn("create index for segment fail", zap.Int64("segID", segment.ID),
zap.Int64("indexID", index.IndexID))
return err
}
}
}
return nil
}
func (s *Server) createIndexForSegmentLoop(ctx context.Context) {
log.Info("start create index for segment loop...")
defer s.serverLoopWg.Done()
ticker := time.NewTicker(time.Minute)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
log.Warn("DataCoord context done, exit...")
return
case <-ticker.C:
segments := s.meta.GetHasUnindexTaskSegments()
for _, segment := range segments {
if err := s.createIndexesForSegment(segment); err != nil {
log.Warn("create index for segment fail, wait for retry", zap.Int64("segID", segment.ID))
continue
}
}
case collID := <-s.notifyIndexChan:
log.Info("receive create index notify", zap.Int64("collID", collID))
segments := s.meta.SelectSegments(func(info *SegmentInfo) bool {
return isFlush(info) && collID == info.CollectionID
})
for _, segment := range segments {
if err := s.createIndexesForSegment(segment); err != nil {
log.Warn("create index for segment fail, wait for retry", zap.Int64("segID", segment.ID))
continue
}
}
case segID := <-s.buildIndexCh:
log.Info("receive new flushed segment", zap.Int64("segID", segID))
segment := s.meta.GetSegmentUnsafe(segID)
if segment == nil {
log.Warn("segment is not exist, no need to build index", zap.Int64("segID", segID))
continue
}
if err := s.createIndexesForSegment(segment); err != nil {
log.Warn("create index for segment fail, wait for retry", zap.Int64("segID", segment.ID))
continue
}
}
}
}
// CreateIndex create an index on collection.
// Index building is asynchronous, so when an index building request comes, an IndexID is assigned to the task and
// will get all flushed segments from DataCoord and record tasks with these segments. The background process
// indexBuilder will find this task and assign it to IndexNode for execution.
func (s *Server) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
log := log.Ctx(ctx)
log.Info("receive CreateIndex request", zap.Int64("CollectionID", req.GetCollectionID()),
zap.String("IndexName", req.GetIndexName()), zap.Int64("fieldID", req.GetFieldID()),
zap.Any("TypeParams", req.GetTypeParams()),
zap.Any("IndexParams", req.GetIndexParams()))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return errResp, nil
}
indexID, err := s.meta.CanCreateIndex(req)
if err != nil {
log.Error("CreateIndex failed", zap.Error(err))
errResp.Reason = err.Error()
return errResp, nil
}
if indexID == 0 {
indexID, err = s.allocator.allocID(ctx)
if err != nil {
log.Warn("failed to alloc indexID", zap.Error(err))
errResp.Reason = "failed to alloc indexID"
return errResp, nil
}
if getIndexType(req.GetIndexParams()) == diskAnnIndex && !s.indexNodeManager.ClientSupportDisk() {
errMsg := "all IndexNodes do not support disk indexes, please verify"
log.Warn(errMsg)
errResp.Reason = errMsg
return errResp, nil
}
}
index := &model.Index{
CollectionID: req.GetCollectionID(),
FieldID: req.GetFieldID(),
IndexID: indexID,
IndexName: req.GetIndexName(),
TypeParams: req.GetTypeParams(),
IndexParams: req.GetIndexParams(),
CreateTime: req.GetTimestamp(),
IsAutoIndex: req.GetIsAutoIndex(),
UserIndexParams: req.GetUserIndexParams(),
}
// Get flushed segments and create index
err = s.meta.CreateIndex(index)
if err != nil {
log.Error("CreateIndex fail", zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64("fieldID", req.GetFieldID()), zap.String("indexName", req.GetIndexName()), zap.Error(err))
errResp.Reason = err.Error()
return errResp, nil
}
select {
case s.notifyIndexChan <- req.GetCollectionID():
default:
}
log.Info("CreateIndex successfully", zap.Int64("collectionID", req.GetCollectionID()),
zap.String("IndexName", req.GetIndexName()), zap.Int64("fieldID", req.GetFieldID()),
zap.Int64("IndexID", indexID))
errResp.ErrorCode = commonpb.ErrorCode_Success
return errResp, nil
}
// GetIndexState gets the index state of the index name in the request from Proxy.
func (s *Server) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
log := log.Ctx(ctx)
log.Info("receive GetIndexState request", zap.Int64("collectionID", req.CollectionID),
zap.String("indexName", req.IndexName))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return &datapb.GetIndexStateResponse{
Status: errResp,
}, nil
}
indexes := s.meta.GetIndexesForCollection(req.GetCollectionID(), req.GetIndexName())
if len(indexes) == 0 {
errResp.ErrorCode = commonpb.ErrorCode_IndexNotExist
errResp.Reason = fmt.Sprintf("there is no index on collection: %d with the index name: %s", req.CollectionID, req.IndexName)
log.Error("GetIndexState fail", zap.Int64("collectionID", req.CollectionID),
zap.String("indexName", req.IndexName), zap.String("fail reason", errResp.Reason))
return &datapb.GetIndexStateResponse{
Status: errResp,
}, nil
}
if len(indexes) > 1 {
log.Warn(msgAmbiguousIndexName())
errResp.ErrorCode = commonpb.ErrorCode_UnexpectedError
errResp.Reason = msgAmbiguousIndexName()
return &datapb.GetIndexStateResponse{
Status: errResp,
}, nil
}
ret := &datapb.GetIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
State: commonpb.IndexState_Finished,
}
indexInfo := &datapb.IndexInfo{
IndexedRows: 0,
TotalRows: 0,
State: 0,
IndexStateFailReason: "",
}
s.completeIndexInfo(indexInfo, indexes[0], s.meta.SelectSegments(func(info *SegmentInfo) bool {
return isSegmentHealthy(info) && info.CollectionID == req.GetCollectionID()
}))
ret.State = indexInfo.State
ret.FailReason = indexInfo.IndexStateFailReason
log.Info("GetIndexState success", zap.Int64("collectionID", req.GetCollectionID()),
zap.String("IndexName", req.GetIndexName()), zap.String("state", ret.GetState().String()))
return ret, nil
}
func (s *Server) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
log := log.Ctx(ctx)
log.Info("receive GetSegmentIndexState", zap.Int64("CollectionID", req.GetCollectionID()),
zap.String("IndexName", req.GetIndexName()), zap.Int64s("fieldID", req.GetSegmentIDs()))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return &datapb.GetSegmentIndexStateResponse{
Status: errResp,
}, nil
}
ret := &datapb.GetSegmentIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
States: make([]*datapb.SegmentIndexState, 0),
}
indexID2CreateTs := s.meta.GetIndexIDByName(req.GetCollectionID(), req.GetIndexName())
if len(indexID2CreateTs) == 0 {
errMsg := fmt.Sprintf("there is no index on collection: %d with the index name: %s", req.CollectionID, req.GetIndexName())
log.Error("GetSegmentIndexState fail", zap.Int64("collectionID", req.GetCollectionID()),
zap.String("indexName", req.GetIndexName()), zap.String("fail reason", errMsg))
return &datapb.GetSegmentIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_IndexNotExist,
Reason: errMsg,
},
}, nil
}
for _, segID := range req.SegmentIDs {
state := s.meta.GetSegmentIndexState(req.GetCollectionID(), segID)
ret.States = append(ret.States, &datapb.SegmentIndexState{
SegmentID: segID,
State: state.state,
FailReason: state.failReason,
})
}
log.Info("GetSegmentIndexState successfully", zap.Int64("collectionID", req.GetCollectionID()),
zap.String("indexName", req.GetIndexName()))
return ret, nil
}
// completeIndexInfo get the building index row count and index task state
func (s *Server) completeIndexInfo(indexInfo *datapb.IndexInfo, index *model.Index, segments []*SegmentInfo) {
var (
cntNone = 0
cntUnissued = 0
cntInProgress = 0
cntFinished = 0
cntFailed = 0
failReason string
totalRows = int64(0)
indexedRows = int64(0)
)
for _, seg := range segments {
totalRows += seg.NumOfRows
segIdx, ok := seg.segmentIndexes[index.IndexID]
if !ok {
if seg.LastExpireTime <= index.CreateTime {
cntUnissued++
}
continue
}
if segIdx.CreateTime > index.CreateTime {
continue
}
switch segIdx.IndexState {
case commonpb.IndexState_IndexStateNone:
// can't to here
cntNone++
case commonpb.IndexState_Unissued:
cntUnissued++
case commonpb.IndexState_InProgress:
cntInProgress++
case commonpb.IndexState_Finished:
cntFinished++
indexedRows += seg.NumOfRows
case commonpb.IndexState_Failed:
cntFailed++
failReason += fmt.Sprintf("%d: %s;", segIdx.SegmentID, segIdx.FailReason)
}
}
allCnt := len(segments)
indexInfo.TotalRows = totalRows
indexInfo.IndexedRows = indexedRows
switch {
case cntFailed > 0:
indexInfo.State = commonpb.IndexState_Failed
indexInfo.IndexStateFailReason = failReason
case cntFinished == allCnt:
indexInfo.State = commonpb.IndexState_Finished
default:
indexInfo.State = commonpb.IndexState_InProgress
}
log.Info("completeIndexInfo success", zap.Int64("collID", index.CollectionID), zap.Int64("indexID", index.IndexID),
zap.Int64("totalRows", indexInfo.TotalRows), zap.Int64("indexRows", indexInfo.IndexedRows),
zap.String("state", indexInfo.State.String()), zap.String("failReason", indexInfo.IndexStateFailReason))
}
// GetIndexBuildProgress get the index building progress by num rows.
func (s *Server) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
log := log.Ctx(ctx)
log.Info("receive GetIndexBuildProgress request", zap.Int64("collID", req.GetCollectionID()),
zap.String("indexName", req.GetIndexName()))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return &datapb.GetIndexBuildProgressResponse{
Status: errResp,
}, nil
}
indexes := s.meta.GetIndexesForCollection(req.GetCollectionID(), req.GetIndexName())
if len(indexes) == 0 {
errMsg := fmt.Sprintf("there is no index on collection: %d with the index name: %s", req.CollectionID, req.IndexName)
log.Error("GetIndexBuildProgress fail", zap.Int64("collectionID", req.CollectionID),
zap.String("indexName", req.IndexName), zap.String("fail reason", errMsg))
return &datapb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_IndexNotExist,
Reason: errMsg,
},
}, nil
}
indexInfo := &datapb.IndexInfo{
IndexedRows: 0,
TotalRows: 0,
State: 0,
}
s.completeIndexInfo(indexInfo, indexes[0], s.meta.SelectSegments(func(info *SegmentInfo) bool {
return isSegmentHealthy(info) && info.CollectionID == req.GetCollectionID()
}))
log.Info("GetIndexBuildProgress success", zap.Int64("collectionID", req.GetCollectionID()),
zap.String("indexName", req.GetIndexName()))
return &datapb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexedRows: indexInfo.IndexedRows,
TotalRows: indexInfo.TotalRows,
}, nil
}
// DescribeIndex describe the index info of the collection.
func (s *Server) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
log := log.Ctx(ctx)
log.Info("receive DescribeIndex request", zap.Int64("collID", req.GetCollectionID()),
zap.String("indexName", req.GetIndexName()))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return &datapb.DescribeIndexResponse{
Status: errResp,
}, nil
}
indexes := s.meta.GetIndexesForCollection(req.GetCollectionID(), req.GetIndexName())
if len(indexes) == 0 {
errMsg := fmt.Sprintf("there is no index on collection: %d with the index name: %s", req.CollectionID, req.IndexName)
log.Error("DescribeIndex fail", zap.Int64("collectionID", req.CollectionID),
zap.String("indexName", req.IndexName), zap.String("fail reason", errMsg))
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_IndexNotExist,
Reason: fmt.Sprint("index doesn't exist, collectionID ", req.CollectionID),
},
}, nil
}
// The total rows of all indexes should be based on the current perspective
segments := s.meta.SelectSegments(func(info *SegmentInfo) bool {
return isFlush(info) && info.CollectionID == req.GetCollectionID()
})
indexInfos := make([]*datapb.IndexInfo, 0)
for _, index := range indexes {
indexInfo := &datapb.IndexInfo{
CollectionID: index.CollectionID,
FieldID: index.FieldID,
IndexName: index.IndexName,
IndexID: index.IndexID,
TypeParams: index.TypeParams,
IndexParams: index.IndexParams,
IndexedRows: 0,
TotalRows: 0,
State: 0,
IndexStateFailReason: "",
IsAutoIndex: index.IsAutoIndex,
UserIndexParams: index.UserIndexParams,
}
s.completeIndexInfo(indexInfo, index, segments)
indexInfos = append(indexInfos, indexInfo)
}
log.Info("DescribeIndex success", zap.Int64("collectionID", req.GetCollectionID()),
zap.String("indexName", req.GetIndexName()))
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: indexInfos,
}, nil
}
// DropIndex deletes indexes based on IndexName. One IndexName corresponds to the index of an entire column. A column is
// divided into many segments, and each segment corresponds to an IndexBuildID. DataCoord uses IndexBuildID to record
// index tasks.
func (s *Server) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
log := log.Ctx(ctx)
log.Info("receive DropIndex request", zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64s("partitionIDs", req.GetPartitionIDs()), zap.String("indexName", req.GetIndexName()),
zap.Bool("drop all indexes", req.GetDropAll()))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return errResp, nil
}
ret := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}
indexes := s.meta.GetIndexesForCollection(req.GetCollectionID(), req.GetIndexName())
if len(indexes) == 0 {
log.Info(fmt.Sprintf("there is no index on collection: %d with the index name: %s", req.CollectionID, req.IndexName))
return ret, nil
}
if !req.GetDropAll() && len(indexes) > 1 {
log.Warn(msgAmbiguousIndexName())
ret.ErrorCode = commonpb.ErrorCode_UnexpectedError
ret.Reason = msgAmbiguousIndexName()
return ret, nil
}
indexIDs := make([]UniqueID, 0)
for _, index := range indexes {
indexIDs = append(indexIDs, index.IndexID)
}
if len(req.GetPartitionIDs()) == 0 {
// drop collection index
err := s.meta.MarkIndexAsDeleted(req.CollectionID, indexIDs)
if err != nil {
log.Error("DropIndex fail", zap.Int64("collectionID", req.CollectionID),
zap.String("indexName", req.IndexName), zap.Error(err))
ret.ErrorCode = commonpb.ErrorCode_UnexpectedError
ret.Reason = err.Error()
return ret, nil
}
}
log.Info("DropIndex success", zap.Int64("collID", req.CollectionID),
zap.Int64s("partitionIDs", req.PartitionIDs), zap.String("indexName", req.IndexName),
zap.Int64s("indexIDs", indexIDs))
return ret, nil
}
// GetIndexInfos gets the index file paths for segment from DataCoord.
func (s *Server) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
log := log.Ctx(ctx)
log.Info("receive GetIndexInfos request", zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64s("segmentIDs", req.GetSegmentIDs()), zap.String("indexName", req.GetIndexName()))
errResp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "",
}
if s.isClosed() {
log.Warn(msgDataCoordIsUnhealthy(paramtable.GetNodeID()))
errResp.ErrorCode = commonpb.ErrorCode_DataCoordNA
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return &datapb.GetIndexInfoResponse{
Status: errResp,
}, nil
}
ret := &datapb.GetIndexInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
SegmentInfo: map[int64]*datapb.SegmentIndexInfo{},
}
for _, segID := range req.SegmentIDs {
segIdxes := s.meta.GetSegmentIndexes(segID)
ret.SegmentInfo[segID] = &datapb.SegmentIndexInfo{
CollectionID: req.CollectionID,
SegmentID: segID,
EnableIndex: false,
IndexInfos: make([]*datapb.IndexFilePathInfo, 0),
}
if len(segIdxes) != 0 {
ret.SegmentInfo[segID].EnableIndex = true
for _, segIdx := range segIdxes {
if segIdx.IndexState == commonpb.IndexState_Finished {
indexFilePaths := metautil.BuildSegmentIndexFilePaths(s.meta.chunkManager.RootPath(), segIdx.BuildID, segIdx.IndexVersion,
segIdx.PartitionID, segIdx.SegmentID, segIdx.IndexFileKeys)
ret.SegmentInfo[segID].IndexInfos = append(ret.SegmentInfo[segID].IndexInfos,
&datapb.IndexFilePathInfo{
SegmentID: segID,
FieldID: s.meta.GetFieldIDByIndexID(segIdx.CollectionID, segIdx.IndexID),
IndexID: segIdx.IndexID,
BuildID: segIdx.BuildID,
IndexName: s.meta.GetIndexNameByID(segIdx.CollectionID, segIdx.IndexID),
IndexParams: s.meta.GetIndexParams(segIdx.CollectionID, segIdx.IndexID),
IndexFilePaths: indexFilePaths,
SerializedSize: segIdx.IndexSize,
IndexVersion: segIdx.IndexVersion,
NumRows: segIdx.NumRows,
})
}
}
}
}
log.Info("GetIndexInfos successfully", zap.Int64("collectionID", req.CollectionID),
zap.String("indexName", req.GetIndexName()))
return ret, nil
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,255 @@
// 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 datacoord
import (
"context"
"sync"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
grpcindexnodeclient "github.com/milvus-io/milvus/internal/distributed/indexnode/client"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
)
// IndexNodeManager is used to manage the client of IndexNode.
type IndexNodeManager struct {
nodeClients map[UniqueID]types.IndexNode
stoppingNodes map[UniqueID]struct{}
lock sync.RWMutex
ctx context.Context
}
// NewNodeManager is used to create a new IndexNodeManager.
func NewNodeManager(ctx context.Context) *IndexNodeManager {
return &IndexNodeManager{
nodeClients: make(map[UniqueID]types.IndexNode),
stoppingNodes: make(map[UniqueID]struct{}),
lock: sync.RWMutex{},
ctx: ctx,
}
}
// setClient sets IndexNode client to node manager.
func (nm *IndexNodeManager) setClient(nodeID UniqueID, client types.IndexNode) {
log.Debug("set IndexNode client", zap.Int64("nodeID", nodeID))
nm.lock.Lock()
defer nm.lock.Unlock()
nm.nodeClients[nodeID] = client
log.Debug("IndexNode IndexNodeManager setClient success", zap.Int64("nodeID", nodeID), zap.Int("IndexNode num", len(nm.nodeClients)))
}
// RemoveNode removes the unused client of IndexNode.
func (nm *IndexNodeManager) RemoveNode(nodeID UniqueID) {
log.Debug("remove IndexNode", zap.Int64("nodeID", nodeID))
nm.lock.Lock()
defer nm.lock.Unlock()
delete(nm.nodeClients, nodeID)
delete(nm.stoppingNodes, nodeID)
metrics.IndexCoordIndexNodeNum.WithLabelValues().Dec()
}
func (nm *IndexNodeManager) StoppingNode(nodeID UniqueID) {
log.Info("IndexCoord", zap.Any("Stopping node with ID", nodeID))
nm.lock.Lock()
defer nm.lock.Unlock()
nm.stoppingNodes[nodeID] = struct{}{}
}
// AddNode adds the client of IndexNode.
func (nm *IndexNodeManager) AddNode(nodeID UniqueID, address string) error {
log.Debug("add IndexNode", zap.Any("nodeID", nodeID), zap.Any("node address", address))
var (
nodeClient types.IndexNode
err error
)
nodeClient, err = grpcindexnodeclient.NewClient(context.TODO(), address, Params.IndexCoordCfg.WithCredential.GetAsBool())
if err != nil {
log.Error("create IndexNode client fail", zap.Error(err))
return err
}
metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc()
nm.setClient(nodeID, nodeClient)
return nil
}
// PeekClient peeks the client with the least load.
func (nm *IndexNodeManager) PeekClient(meta *model.SegmentIndex) (UniqueID, types.IndexNode) {
allClients := nm.GetAllClients()
if len(allClients) == 0 {
log.Error("there is no IndexNode online")
return -1, nil
}
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
ctx, cancel := context.WithCancel(nm.ctx)
var (
peekNodeID = UniqueID(0)
nodeMutex = sync.Mutex{}
wg = sync.WaitGroup{}
)
for nodeID, client := range allClients {
nodeID := nodeID
client := client
wg.Add(1)
go func() {
defer wg.Done()
resp, err := client.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
zap.String("reason", resp.Status.Reason))
return
}
if resp.TaskSlots > 0 {
nodeMutex.Lock()
defer nodeMutex.Unlock()
log.Info("peek client success", zap.Int64("nodeID", nodeID))
if peekNodeID == 0 {
peekNodeID = nodeID
}
cancel()
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
return
}
}()
}
wg.Wait()
cancel()
if peekNodeID != 0 {
log.Info("peek client success", zap.Int64("nodeID", peekNodeID))
return peekNodeID, allClients[peekNodeID]
}
log.RatedDebug(5, "peek client fail")
return 0, nil
}
func (nm *IndexNodeManager) ClientSupportDisk() bool {
log.Info("check if client support disk index")
allClients := nm.GetAllClients()
if len(allClients) == 0 {
log.Warn("there is no IndexNode online")
return false
}
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
ctx, cancel := context.WithCancel(nm.ctx)
var (
enableDisk = false
nodeMutex = sync.Mutex{}
wg = sync.WaitGroup{}
)
for nodeID, client := range allClients {
nodeID := nodeID
client := client
wg.Add(1)
go func() {
defer wg.Done()
resp, err := client.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
zap.String("reason", resp.Status.Reason))
return
}
log.Debug("get job stats success", zap.Int64("nodeID", nodeID), zap.Bool("enable disk", resp.EnableDisk))
if resp.EnableDisk {
nodeMutex.Lock()
defer nodeMutex.Unlock()
cancel()
if !enableDisk {
enableDisk = true
}
return
}
}()
}
wg.Wait()
cancel()
if enableDisk {
log.Info("IndexNode support disk index")
return true
}
log.Error("all IndexNodes do not support disk indexes")
return false
}
func (nm *IndexNodeManager) GetAllClients() map[UniqueID]types.IndexNode {
nm.lock.RLock()
defer nm.lock.RUnlock()
allClients := make(map[UniqueID]types.IndexNode, len(nm.nodeClients))
for nodeID, client := range nm.nodeClients {
if _, ok := nm.stoppingNodes[nodeID]; !ok {
allClients[nodeID] = client
}
}
return allClients
}
func (nm *IndexNodeManager) GetClientByID(nodeID UniqueID) (types.IndexNode, bool) {
nm.lock.RLock()
defer nm.lock.RUnlock()
client, ok := nm.nodeClients[nodeID]
return client, ok
}
// indexNodeGetMetricsResponse record the metrics information of IndexNode.
type indexNodeGetMetricsResponse struct {
resp *milvuspb.GetMetricsResponse
err error
}
// getMetrics get metrics information of all IndexNode.
func (nm *IndexNodeManager) getMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) []indexNodeGetMetricsResponse {
var clients []types.IndexNode
nm.lock.RLock()
for _, node := range nm.nodeClients {
clients = append(clients, node)
}
nm.lock.RUnlock()
ret := make([]indexNodeGetMetricsResponse, 0, len(nm.nodeClients))
for _, node := range clients {
resp, err := node.GetMetrics(ctx, req)
ret = append(ret, indexNodeGetMetricsResponse{
resp: resp,
err: err,
})
}
return ret
}

View File

@ -0,0 +1,270 @@
// 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 datacoord
import (
"context"
"errors"
"sync"
"testing"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/stretchr/testify/assert"
)
func TestIndexNodeManager_AddNode(t *testing.T) {
nm := NewNodeManager(context.Background())
nodeID, client := nm.PeekClient(&model.SegmentIndex{})
assert.Equal(t, int64(-1), nodeID)
assert.Nil(t, client)
t.Run("success", func(t *testing.T) {
err := nm.AddNode(1, "indexnode-1")
assert.Nil(t, err)
})
t.Run("fail", func(t *testing.T) {
err := nm.AddNode(2, "")
assert.Error(t, err)
})
}
func TestIndexNodeManager_PeekClient(t *testing.T) {
t.Run("multiple unavailable IndexNode", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.TODO(),
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
2: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
3: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
4: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
5: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
},
6: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
},
7: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
},
8: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
TaskSlots: 1,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
},
},
9: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
TaskSlots: 10,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
},
},
},
}
nodeID, client := nm.PeekClient(&model.SegmentIndex{})
assert.NotNil(t, client)
assert.Contains(t, []UniqueID{8, 9}, nodeID)
})
}
func TestIndexNodeManager_ClientSupportDisk(t *testing.T) {
t.Run("support", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
TaskSlots: 1,
JobInfos: nil,
EnableDisk: true,
}, nil
},
},
},
}
support := nm.ClientSupportDisk()
assert.True(t, support)
})
t.Run("not support", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
TaskSlots: 1,
JobInfos: nil,
EnableDisk: false,
}, nil
},
},
},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
t.Run("no indexnode", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
t.Run("error", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return nil, errors.New("error")
},
},
},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
t.Run("fail reason", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
TaskSlots: 0,
JobInfos: nil,
EnableDisk: false,
}, nil
},
},
},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
}
func TestNodeManager_StoppingNode(t *testing.T) {
nm := NewNodeManager(context.Background())
err := nm.AddNode(1, "indexnode-1")
assert.NoError(t, err)
assert.Equal(t, 1, len(nm.GetAllClients()))
nm.StoppingNode(1)
assert.Equal(t, 0, len(nm.GetAllClients()))
assert.Equal(t, 1, len(nm.stoppingNodes))
nm.RemoveNode(1)
assert.Equal(t, 0, len(nm.GetAllClients()))
assert.Equal(t, 0, len(nm.stoppingNodes))
}

View File

@ -25,6 +25,8 @@ import (
"time"
"github.com/golang/protobuf/proto"
"github.com/samber/lo"
"go.uber.org/zap"
"golang.org/x/exp/maps"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -34,6 +36,7 @@ import (
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
@ -42,8 +45,6 @@ import (
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/samber/lo"
"go.uber.org/zap"
)
type meta struct {
@ -54,6 +55,13 @@ type meta struct {
segments *SegmentsInfo // segment id to segment info
channelCPs map[string]*internalpb.MsgPosition // vChannel -> channel checkpoint/see position
chunkManager storage.ChunkManager
// collectionIndexes records which indexes are on the collection
// collID -> indexID -> index
indexes map[UniqueID]map[UniqueID]*model.Index
// buildID2Meta records the meta information of the segment
// buildID -> segmentIndex
buildID2SegmentIndex map[UniqueID]*model.SegmentIndex
}
// A local cache of segment metric update. Must call commit() to take effect.
@ -74,12 +82,14 @@ type collectionInfo struct {
// NewMeta creates meta from provided `kv.TxnKV`
func newMeta(ctx context.Context, kv kv.TxnKV, chunkManagerRootPath string, chunkManager storage.ChunkManager) (*meta, error) {
mt := &meta{
ctx: ctx,
catalog: &datacoord.Catalog{Txn: kv, ChunkManagerRootPath: chunkManagerRootPath},
collections: make(map[UniqueID]*collectionInfo),
segments: NewSegmentsInfo(),
channelCPs: make(map[string]*internalpb.MsgPosition),
chunkManager: chunkManager,
ctx: ctx,
catalog: &datacoord.Catalog{Txn: kv, ChunkManagerRootPath: chunkManagerRootPath},
collections: make(map[UniqueID]*collectionInfo),
segments: NewSegmentsInfo(),
channelCPs: make(map[string]*internalpb.MsgPosition),
chunkManager: chunkManager,
indexes: make(map[UniqueID]map[UniqueID]*model.Index),
buildID2SegmentIndex: make(map[UniqueID]*model.SegmentIndex),
}
err := mt.reloadFromKV()
if err != nil {
@ -120,6 +130,25 @@ func (m *meta) reloadFromKV() error {
for vChannel, pos := range channelCPs {
m.channelCPs[vChannel] = pos
}
// load field indexes
fieldIndexes, err := m.catalog.ListIndexes(m.ctx)
if err != nil {
log.Error("DataCoord meta reloadFromKV load field indexes fail", zap.Error(err))
return err
}
for _, fieldIndex := range fieldIndexes {
m.updateCollectionIndex(fieldIndex)
}
segmentIndexes, err := m.catalog.ListSegmentIndexes(m.ctx)
if err != nil {
log.Error("DataCoord meta reloadFromKV load segment indexes fail", zap.Error(err))
return err
}
for _, segIdx := range segmentIndexes {
m.updateSegmentIndex(segIdx)
}
record.Record("meta reloadFromKV")
return nil
}

View File

@ -19,24 +19,22 @@ package datacoord
import (
"context"
"fmt"
"path/filepath"
"strconv"
"strings"
"testing"
"github.com/milvus-io/milvus/internal/common"
"github.com/samber/lo"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/kv"
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util"
"github.com/samber/lo"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
type mockEtcdKv struct {
@ -63,6 +61,42 @@ func (mek *mockEtcdKv) LoadWithPrefix(key string) ([]string, []string, error) {
Timestamp: 1000,
}
val, _ = proto.Marshal(channelCP)
case strings.Contains(key, util.FieldIndexPrefix):
index := &datapb.FieldIndex{
IndexInfo: &datapb.IndexInfo{
CollectionID: 0,
FieldID: 1,
IndexName: "_default_idx_101",
IndexID: 0,
TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "128"}},
IndexParams: []*commonpb.KeyValuePair{{Key: "index_type", Value: "HNSW"}},
IsAutoIndex: false,
UserIndexParams: nil,
},
Deleted: false,
CreateTime: 0,
}
val, _ = proto.Marshal(index)
case strings.Contains(key, util.SegmentIndexPrefix):
segIndex := &datapb.SegmentIndex{
CollectionID: 0,
PartitionID: 0,
SegmentID: 1,
NumRows: 1025,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexVersion: 0,
State: commonpb.IndexState_Unissued,
FailReason: "",
IndexFileKeys: nil,
Deleted: false,
CreateTime: 0,
SerializeSize: 0,
WriteHandoff: false,
}
val, _ = proto.Marshal(segIndex)
default:
return nil, nil, fmt.Errorf("invalid key")
}
@ -70,6 +104,22 @@ func (mek *mockEtcdKv) LoadWithPrefix(key string) ([]string, []string, error) {
return nil, []string{string(val)}, nil
}
func (mek *mockEtcdKv) Save(key, value string) error {
return nil
}
func (mek *mockEtcdKv) MultiSave(kvs map[string]string) error {
return nil
}
func (mek *mockEtcdKv) Remove(key string) error {
return nil
}
func (mek *mockEtcdKv) MultiRemove(keys []string) error {
return nil
}
type mockKvLoadSegmentError struct {
kv.TxnKV
}
@ -185,6 +235,31 @@ func (mek *mockKvIllegalStatslog) LoadWithPrefix(key string) ([]string, []string
return nil, []string{string(val)}, nil
}
type mockLoadIndexError struct {
kv.TxnKV
}
func (mek *mockLoadIndexError) LoadWithPrefix(key string) ([]string, []string, error) {
switch {
case strings.Contains(key, util.FieldIndexPrefix):
return nil, nil, fmt.Errorf("LoadWithPrefix for index error")
}
return nil, nil, nil
}
type mockLoadSegmentIndexError struct {
kv.TxnKV
}
func (mek *mockLoadSegmentIndexError) LoadWithPrefix(key string) ([]string, []string, error) {
switch {
case strings.Contains(key, util.SegmentIndexPrefix):
return nil, nil, fmt.Errorf("LoadWithPrefix for segment index error")
}
return nil, nil, nil
}
func TestMetaReloadFromKV(t *testing.T) {
t.Run("Test ReloadFromKV success", func(t *testing.T) {
fkv := &mockEtcdKv{}
@ -239,6 +314,16 @@ func TestMetaReloadFromKV(t *testing.T) {
_, err := newMeta(context.TODO(), fkv, "", nil)
assert.NotNil(t, err)
})
t.Run("Test ReloadFromKV load index fails", func(t *testing.T) {
fkv := &mockLoadIndexError{}
_, err := newMeta(context.TODO(), fkv, "", nil)
assert.Error(t, err)
})
t.Run("Test ReloadFromKV load segment index fails", func(t *testing.T) {
fkv := &mockLoadSegmentIndexError{}
_, err := newMeta(context.TODO(), fkv, "", nil)
assert.Error(t, err)
})
}
func TestMeta_Basic(t *testing.T) {
@ -613,43 +698,6 @@ func TestUpdateFlushSegmentsInfo(t *testing.T) {
})
}
func TestSaveHandoffMeta(t *testing.T) {
kvClient := memkv.NewMemoryKV()
meta, err := newMeta(context.TODO(), kvClient, "", nil)
assert.Nil(t, err)
info := &datapb.SegmentInfo{
ID: 100,
State: commonpb.SegmentState_Flushing,
}
segmentInfo := &SegmentInfo{
SegmentInfo: info,
}
err = meta.catalog.AddSegment(context.TODO(), segmentInfo.SegmentInfo)
assert.Nil(t, err)
keys, _, err := kvClient.LoadWithPrefix(util.FlushedSegmentPrefix)
assert.Nil(t, err)
assert.Equal(t, 0, len(keys))
newInfo := &datapb.SegmentInfo{
ID: 100,
State: commonpb.SegmentState_Flushed,
}
err = meta.catalog.AlterSegment(context.TODO(), newInfo, segmentInfo.SegmentInfo)
assert.Nil(t, err)
keys, _, err = kvClient.LoadWithPrefix(util.FlushedSegmentPrefix)
assert.Nil(t, err)
assert.Equal(t, 1, len(keys))
segmentID, err := strconv.ParseInt(filepath.Base(keys[0]), 10, 64)
assert.Nil(t, err)
assert.Equal(t, 100, int(segmentID))
}
func TestMeta_alterMetaStore(t *testing.T) {
toAlter := []*datapb.SegmentInfo{
{

View File

@ -28,7 +28,6 @@ import (
"github.com/milvus-io/milvus/internal/kv"
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
@ -112,6 +111,10 @@ func (kv *removeFailKV) MultiRemove(key []string) error {
return errors.New("mocked fail")
}
func (kv *removeFailKV) Remove(key string) error {
return errors.New("mocked fail")
}
func newMockAllocator() *MockAllocator {
return &MockAllocator{}
}
@ -801,7 +804,7 @@ func (m *mockIndexCoord) Start() error {
return nil
}
func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
if req.CollectionID == 10000 {
return nil, errors.New("server down")
}
@ -809,11 +812,11 @@ func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.Describ
// Has diskann index
if req.CollectionID == 1000 || req.CollectionID == 2000 ||
req.CollectionID == 3000 || req.CollectionID == 4000 {
return &indexpb.DescribeIndexResponse{
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
IndexInfos: []*datapb.IndexInfo{
{
CollectionID: req.CollectionID,
FieldID: 0,
@ -832,11 +835,11 @@ func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.Describ
}
// Has common index
return &indexpb.DescribeIndexResponse{
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
IndexInfos: []*datapb.IndexInfo{
{
CollectionID: 1,
FieldID: 0,
@ -849,17 +852,17 @@ func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.Describ
}, nil
}
func (m *mockIndexCoord) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
func (m *mockIndexCoord) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
segmentID := req.GetSegmentIDs()[0]
collectionID := req.GetCollectionID()
return &indexpb.GetIndexInfoResponse{
return &datapb.GetIndexInfoResponse{
Status: &commonpb.Status{},
SegmentInfo: map[int64]*indexpb.SegmentInfo{
SegmentInfo: map[int64]*datapb.SegmentIndexInfo{
segmentID: {
EnableIndex: true,
CollectionID: collectionID,
SegmentID: segmentID,
IndexInfos: []*indexpb.IndexFilePathInfo{
IndexInfos: []*datapb.IndexFilePathInfo{
{
FieldID: int64(201),
},

View File

@ -19,6 +19,8 @@ package datacoord
import (
"time"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -33,10 +35,11 @@ type SegmentsInfo struct {
// SegmentInfo wraps datapb.SegmentInfo and patches some extra info on it
type SegmentInfo struct {
*datapb.SegmentInfo
currRows int64
allocations []*Allocation
lastFlushTime time.Time
isCompacting bool
segmentIndexes map[UniqueID]*model.SegmentIndex
currRows int64
allocations []*Allocation
lastFlushTime time.Time
isCompacting bool
// a cache to avoid calculate twice
size int64
lastWrittenTime time.Time
@ -48,10 +51,11 @@ type SegmentInfo struct {
// the worst case scenario is to have a segment with twice size we expects
func NewSegmentInfo(info *datapb.SegmentInfo) *SegmentInfo {
return &SegmentInfo{
SegmentInfo: info,
currRows: info.GetNumOfRows(),
allocations: make([]*Allocation, 0, 16),
lastFlushTime: time.Now().Add(-1 * flushInterval),
SegmentInfo: info,
segmentIndexes: make(map[UniqueID]*model.SegmentIndex),
currRows: info.GetNumOfRows(),
allocations: make([]*Allocation, 0, 16),
lastFlushTime: time.Now().Add(-1 * flushInterval),
// A growing segment from recovery can be also considered idle.
lastWrittenTime: getZeroTime(),
}
@ -93,6 +97,20 @@ func (s *SegmentsInfo) SetSegment(segmentID UniqueID, segment *SegmentInfo) {
s.segments[segmentID] = segment
}
// SetSegmentIndex sets SegmentIndex with segmentID, perform overwrite if already exists
func (s *SegmentsInfo) SetSegmentIndex(segmentID UniqueID, segIndex *model.SegmentIndex) {
if s.segments[segmentID].segmentIndexes == nil {
s.segments[segmentID].segmentIndexes = make(map[UniqueID]*model.SegmentIndex)
}
s.segments[segmentID].segmentIndexes[segIndex.IndexID] = segIndex
}
func (s *SegmentsInfo) DropSegmentIndex(segmentID UniqueID, indexID UniqueID) {
if _, ok := s.segments[segmentID]; ok {
delete(s.segments[segmentID].segmentIndexes, indexID)
}
}
// SetRowCount sets rowCount info for SegmentInfo with provided segmentID
// if SegmentInfo not found, do nothing
func (s *SegmentsInfo) SetRowCount(segmentID UniqueID, rowCount int64) {
@ -196,12 +214,17 @@ func (s *SegmentsInfo) SetIsCompacting(segmentID UniqueID, isCompacting bool) {
// Clone deep clone the segment info and return a new instance
func (s *SegmentInfo) Clone(opts ...SegmentInfoOption) *SegmentInfo {
info := proto.Clone(s.SegmentInfo).(*datapb.SegmentInfo)
segmentIndexes := make(map[UniqueID]*model.SegmentIndex, len(s.segmentIndexes))
for indexID, segIdx := range s.segmentIndexes {
segmentIndexes[indexID] = model.CloneSegmentIndex(segIdx)
}
cloned := &SegmentInfo{
SegmentInfo: info,
currRows: s.currRows,
allocations: s.allocations,
lastFlushTime: s.lastFlushTime,
isCompacting: s.isCompacting,
SegmentInfo: info,
segmentIndexes: segmentIndexes,
currRows: s.currRows,
allocations: s.allocations,
lastFlushTime: s.lastFlushTime,
isCompacting: s.isCompacting,
//cannot copy size, since binlog may be changed
lastWrittenTime: s.lastWrittenTime,
}
@ -213,8 +236,13 @@ func (s *SegmentInfo) Clone(opts ...SegmentInfoOption) *SegmentInfo {
// ShadowClone shadow clone the segment and return a new instance
func (s *SegmentInfo) ShadowClone(opts ...SegmentInfoOption) *SegmentInfo {
segmentIndexes := make(map[UniqueID]*model.SegmentIndex, len(s.segmentIndexes))
for indexID, segIdx := range s.segmentIndexes {
segmentIndexes[indexID] = model.CloneSegmentIndex(segIdx)
}
cloned := &SegmentInfo{
SegmentInfo: s.SegmentInfo,
segmentIndexes: segmentIndexes,
currRows: s.currRows,
allocations: s.allocations,
lastFlushTime: s.lastFlushTime,

View File

@ -1,217 +0,0 @@
// 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 datacoord
import (
"path"
"strconv"
"sync"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"go.uber.org/zap"
)
type SegmentReferenceManager struct {
etcdKV kv.BaseKV
// taskID -> (nodeID -> segmentReferenceLock), taskID must be globally unique in a component
segmentsLock map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock
segmentReferCnt map[UniqueID]int
lock sync.RWMutex
}
func NewSegmentReferenceManager(etcdKV kv.BaseKV, onlineIDs []UniqueID) (*SegmentReferenceManager, error) {
log.Info("create a new segment reference manager")
segReferManager := &SegmentReferenceManager{
etcdKV: etcdKV,
segmentsLock: make(map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock),
segmentReferCnt: map[UniqueID]int{},
lock: sync.RWMutex{},
}
_, values, err := segReferManager.etcdKV.LoadWithPrefix(segmentReferPrefix)
if err != nil {
log.Error("load segments lock from etcd failed", zap.Error(err))
return nil, err
}
for _, value := range values {
segReferLock := &datapb.SegmentReferenceLock{}
if err = proto.Unmarshal([]byte(value), segReferLock); err != nil {
log.Error("unmarshal segment reference lock failed", zap.Error(err))
return nil, err
}
if _, ok := segReferManager.segmentsLock[segReferLock.TaskID]; !ok {
segReferManager.segmentsLock[segReferLock.TaskID] = map[UniqueID]*datapb.SegmentReferenceLock{}
}
segReferManager.segmentsLock[segReferLock.TaskID][segReferLock.NodeID] = segReferLock
for _, segID := range segReferLock.SegmentIDs {
segReferManager.segmentReferCnt[segID]++
}
}
err = segReferManager.recoverySegReferManager(onlineIDs)
if err != nil {
log.Error("recovery segment reference manager failed", zap.Error(err))
return nil, err
}
log.Info("create new segment reference manager successfully")
return segReferManager, nil
}
func generateLocKey(taskID, nodeID UniqueID) string {
return path.Join(segmentReferPrefix, strconv.FormatInt(taskID, 10), strconv.FormatInt(nodeID, 10))
}
// AddSegmentsLock adds a reference lock on segments to ensure the segments does not compaction during the reference period.
func (srm *SegmentReferenceManager) AddSegmentsLock(taskID int64, segIDs []UniqueID, nodeID UniqueID) error {
srm.lock.Lock()
defer srm.lock.Unlock()
log.Info("add reference lock on segments", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID))
segReferLock := &datapb.SegmentReferenceLock{
TaskID: taskID,
NodeID: nodeID,
SegmentIDs: segIDs,
}
value, err := proto.Marshal(segReferLock)
if err != nil {
log.Error("AddSegmentsLock marshal failed", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID),
zap.Int64s("segIDs", segIDs), zap.Error(err))
return err
}
if err = srm.etcdKV.Save(generateLocKey(taskID, nodeID), string(value)); err != nil {
log.Error("AddSegmentsLock save segment lock to etcd failed", zap.Int64("taskID", taskID),
zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(err))
return err
}
if _, ok := srm.segmentsLock[taskID]; !ok {
srm.segmentsLock[taskID] = map[UniqueID]*datapb.SegmentReferenceLock{}
}
srm.segmentsLock[taskID][nodeID] = segReferLock
for _, segID := range segIDs {
srm.segmentReferCnt[segID]++
}
log.Info("add reference lock on segments successfully", zap.Int64("taskID", taskID), zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID))
return nil
}
func (srm *SegmentReferenceManager) ReleaseSegmentsLock(taskID int64, nodeID UniqueID) error {
srm.lock.Lock()
defer srm.lock.Unlock()
log.Info("release reference lock by taskID", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
if _, ok := srm.segmentsLock[taskID]; !ok {
log.Warn("taskID has no reference lock on segment", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return nil
}
if _, ok := srm.segmentsLock[taskID][nodeID]; !ok {
log.Warn("taskID has no reference lock on segment with the nodeID", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return nil
}
if err := srm.etcdKV.Remove(generateLocKey(taskID, nodeID)); err != nil {
log.Error("remove reference lock paths by taskID failed", zap.Int64("taskID", taskID),
zap.Int64("nodeID", nodeID), zap.Error(err))
return err
}
for _, segID := range srm.segmentsLock[taskID][nodeID].SegmentIDs {
srm.segmentReferCnt[segID]--
if srm.segmentReferCnt[segID] <= 0 {
delete(srm.segmentReferCnt, segID)
}
}
delete(srm.segmentsLock[taskID], nodeID)
if len(srm.segmentsLock[taskID]) == 0 {
delete(srm.segmentsLock, taskID)
}
log.Info("release reference lock by taskID successfully", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return nil
}
func (srm *SegmentReferenceManager) ReleaseSegmentsLockByNodeID(nodeID UniqueID) error {
srm.lock.Lock()
defer srm.lock.Unlock()
log.Info("release reference lock on segments by node", zap.Int64("nodeID", nodeID))
for taskID, segReferLock := range srm.segmentsLock {
if _, ok := segReferLock[nodeID]; !ok {
continue
}
// The reason for not using MultiRemove is to prevent too many keys.
if err := srm.etcdKV.Remove(generateLocKey(taskID, nodeID)); err != nil {
log.Warn("remove reference lock path by taskID failed, need to retry", zap.Int64("nodeID", nodeID),
zap.Int64("taskID", taskID), zap.Error(err))
return err
}
for _, segID := range segReferLock[nodeID].SegmentIDs {
srm.segmentReferCnt[segID]--
if srm.segmentReferCnt[segID] <= 0 {
delete(srm.segmentReferCnt, segID)
}
}
delete(srm.segmentsLock[taskID], nodeID)
if len(srm.segmentsLock[taskID]) == 0 {
delete(srm.segmentsLock, taskID)
}
}
log.Info("release reference lock on segments by node successfully", zap.Int64("nodeID", nodeID))
return nil
}
func (srm *SegmentReferenceManager) recoverySegReferManager(nodeIDs []UniqueID) error {
log.Info("recovery reference lock on segments by online nodes", zap.Int64s("online nodeIDs", nodeIDs))
onlineIDs := make(map[UniqueID]struct{})
for _, nodeID := range nodeIDs {
onlineIDs[nodeID] = struct{}{}
}
offlineIDs := make(map[UniqueID]struct{})
for _, segLock := range srm.segmentsLock {
for nodeID := range segLock {
if _, ok := onlineIDs[nodeID]; !ok {
offlineIDs[nodeID] = struct{}{}
}
}
}
for nodeID := range offlineIDs {
if err := srm.ReleaseSegmentsLockByNodeID(nodeID); err != nil {
log.Error("remove reference lock on segments by offline node failed",
zap.Int64("offline nodeID", nodeID), zap.Error(err))
return err
}
}
log.Info("recovery reference lock on segments by online nodes successfully", zap.Int64s("online nodeIDs", nodeIDs),
zap.Any("offline nodeIDs", offlineIDs))
return nil
}
func (srm *SegmentReferenceManager) HasSegmentLock(segID UniqueID) bool {
srm.lock.RLock()
defer srm.lock.RUnlock()
if _, ok := srm.segmentReferCnt[segID]; !ok {
return false
}
return true
}

View File

@ -1,343 +0,0 @@
// 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 datacoord
import (
"errors"
"path"
"strconv"
"testing"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/kv"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/stretchr/testify/assert"
)
func Test_SegmentReferenceManager(t *testing.T) {
var segRefer *SegmentReferenceManager
var err error
Params.Init()
etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
Params.EtcdCfg.Endpoints.GetAsStrings(),
Params.EtcdCfg.EtcdTLSCert.GetValue(),
Params.EtcdCfg.EtcdTLSKey.GetValue(),
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, "unittest")
t.Run("NewSegmentReferenceManager", func(t *testing.T) {
var segRefer *SegmentReferenceManager
var err error
var locKey string
nodeID := int64(1)
taskID := int64(10)
locKey = path.Join(segmentReferPrefix, strconv.FormatInt(taskID, 10))
segReferLock1 := &datapb.SegmentReferenceLock{
TaskID: taskID,
NodeID: nodeID,
SegmentIDs: []UniqueID{1},
}
value, err := proto.Marshal(segReferLock1)
assert.NoError(t, err)
err = etcdKV.Save(locKey, string(value))
assert.NoError(t, err)
segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID})
assert.NoError(t, err)
assert.NotNil(t, segRefer)
segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID + 1})
assert.NoError(t, err)
assert.NotNil(t, segRefer)
err = etcdKV.Remove(locKey)
assert.NoError(t, err)
locKey = path.Join(segmentReferPrefix, strconv.FormatInt(taskID, 10))
err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10))
assert.NoError(t, err)
segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID})
assert.Error(t, err)
assert.Nil(t, segRefer)
err = etcdKV.Remove(locKey)
assert.NoError(t, err)
})
segIDs := []UniqueID{1, 2, 3, 4, 5}
nodeID := UniqueID(1)
taskID := UniqueID(10)
segRefer, err = NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
var has bool
t.Run("AddSegmentsLock", func(t *testing.T) {
err = segRefer.AddSegmentsLock(taskID, segIDs, nodeID)
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.True(t, has)
}
})
t.Run("ReleaseSegmentsLock", func(t *testing.T) {
err = segRefer.ReleaseSegmentsLock(taskID, nodeID+1)
assert.NoError(t, err)
err = segRefer.ReleaseSegmentsLock(taskID, nodeID)
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.False(t, has)
}
taskID = UniqueID(11)
err = segRefer.ReleaseSegmentsLock(taskID, nodeID)
assert.NoError(t, err)
has = segRefer.HasSegmentLock(6)
assert.False(t, has)
err = segRefer.ReleaseSegmentsLock(taskID, nodeID)
assert.NoError(t, err)
has = segRefer.HasSegmentLock(6)
assert.False(t, has)
})
t.Run("ReleaseSegmentsLockByNodeID", func(t *testing.T) {
segIDs = []UniqueID{10, 11, 12, 13, 14, 15}
nodeID = 2
taskID = UniqueID(12)
err = segRefer.AddSegmentsLock(taskID, segIDs, nodeID)
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.True(t, has)
}
err = segRefer.ReleaseSegmentsLockByNodeID(nodeID)
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.False(t, has)
}
err = segRefer.ReleaseSegmentsLockByNodeID(nodeID)
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.False(t, has)
}
err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(11))
assert.NoError(t, err)
})
t.Run("RecoverySegReferManager", func(t *testing.T) {
segIDs = []UniqueID{16, 17, 18, 19, 20}
taskID = UniqueID(13)
err = segRefer.AddSegmentsLock(taskID, segIDs, UniqueID(3))
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.True(t, has)
}
segIDs2 := []UniqueID{21, 22, 23, 24, 25}
err = segRefer.AddSegmentsLock(taskID, segIDs2, UniqueID(4))
assert.NoError(t, err)
for _, segID := range segIDs2 {
has = segRefer.HasSegmentLock(segID)
assert.True(t, has)
}
err = segRefer.recoverySegReferManager([]int64{4, 5})
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.False(t, has)
}
err = segRefer.ReleaseSegmentsLockByNodeID(4)
assert.NoError(t, err)
for _, segID := range segIDs2 {
has = segRefer.HasSegmentLock(segID)
assert.False(t, has)
}
})
t.Run("HasSegmentLock", func(t *testing.T) {
exist := segRefer.HasSegmentLock(UniqueID(1))
assert.False(t, exist)
})
t.Run("GetHasReferLockSegmentIDs", func(t *testing.T) {
segIDs = []UniqueID{26, 27, 28, 29, 30}
taskID = UniqueID(14)
err = segRefer.AddSegmentsLock(taskID, segIDs, UniqueID(5))
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.True(t, has)
}
err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(5))
assert.NoError(t, err)
for _, segID := range segIDs {
has = segRefer.HasSegmentLock(segID)
assert.False(t, has)
}
})
}
type etcdKVMock struct {
kv.BaseKV
Fail int
}
func (em *etcdKVMock) Save(key, value string) error {
if em.Fail > 0 {
return errors.New("error occurred")
}
return nil
}
func (em *etcdKVMock) Remove(key string) error {
if em.Fail > 0 {
return errors.New("error occurred")
}
return nil
}
func (em *etcdKVMock) LoadWithPrefix(prefix string) ([]string, []string, error) {
if em.Fail > 2 {
return nil, nil, errors.New("error occurs")
}
if em.Fail > 1 {
return []string{"key"}, []string{"value"}, nil
}
referLock := &datapb.SegmentReferenceLock{
TaskID: 1,
NodeID: 1,
SegmentIDs: []UniqueID{1, 2, 3},
}
value, _ := proto.Marshal(referLock)
return []string{segmentReferPrefix + "/1/1"}, []string{string(value)}, nil
}
func TestSegmentReferenceManager_Error(t *testing.T) {
emKV := &etcdKVMock{
Fail: 3,
}
t.Run("NewSegmentReferenceManager", func(t *testing.T) {
segRefer, err := NewSegmentReferenceManager(emKV, nil)
assert.Error(t, err)
assert.Nil(t, segRefer)
emKV2 := &etcdKVMock{Fail: 2}
segRefer, err = NewSegmentReferenceManager(emKV2, nil)
assert.Error(t, err)
assert.Nil(t, segRefer)
emKV3 := &etcdKVMock{Fail: 1}
segRefer, err = NewSegmentReferenceManager(emKV3, nil)
assert.Error(t, err)
assert.Nil(t, segRefer)
})
segRefer := &SegmentReferenceManager{
etcdKV: emKV,
}
taskID := UniqueID(1)
t.Run("AddSegmentsLock", func(t *testing.T) {
err := segRefer.AddSegmentsLock(taskID, []UniqueID{1}, 1)
assert.Error(t, err)
})
t.Run("ReleaseSegmentsLock", func(t *testing.T) {
nodeID := UniqueID(1)
segRefer = &SegmentReferenceManager{
etcdKV: emKV,
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{
taskID: {
nodeID: {
TaskID: taskID,
NodeID: nodeID,
SegmentIDs: []UniqueID{1, 2, 3},
},
},
},
}
err := segRefer.ReleaseSegmentsLock(taskID, 1)
assert.Error(t, err)
})
t.Run("ReleaseSegmentsLockByNodeID", func(t *testing.T) {
nodeID := UniqueID(1)
segRefer = &SegmentReferenceManager{
etcdKV: emKV,
segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{
taskID: {
nodeID: {
TaskID: taskID,
NodeID: nodeID,
SegmentIDs: []UniqueID{1, 2, 3},
},
},
},
}
err := segRefer.ReleaseSegmentsLockByNodeID(nodeID)
assert.Error(t, err)
})
t.Run("recoverySegReferManager", func(t *testing.T) {
segRefer.segmentsLock = map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{
2: {
3: {
TaskID: 2,
NodeID: 3,
SegmentIDs: []UniqueID{1, 2, 3},
},
},
}
err := segRefer.recoverySegReferManager([]UniqueID{1})
assert.Error(t, err)
})
}

View File

@ -115,22 +115,26 @@ type Server struct {
metricsCacheManager *metricsinfo.MetricsCacheManager
flushCh chan UniqueID
factory dependency.Factory
flushCh chan UniqueID
buildIndexCh chan UniqueID
notifyIndexChan chan UniqueID
factory dependency.Factory
session *sessionutil.Session
dnEventCh <-chan *sessionutil.SessionEvent
//icEventCh <-chan *sessionutil.SessionEvent
qcEventCh <-chan *sessionutil.SessionEvent
inEventCh <-chan *sessionutil.SessionEvent
//qcEventCh <-chan *sessionutil.SessionEvent
enableActiveStandBy bool
activateFunc func()
dataNodeCreator dataNodeCreatorFunc
rootCoordClientCreator rootCoordCreatorFunc
indexCoord types.IndexCoord
//indexCoord types.IndexCoord
segReferManager *SegmentReferenceManager
//segReferManager *SegmentReferenceManager
indexBuilder *indexBuilder
indexNodeManager *IndexNodeManager
}
// ServerHelper datacoord server injection helper
@ -190,6 +194,8 @@ func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Optio
quitCh: make(chan struct{}),
factory: factory,
flushCh: make(chan UniqueID, 1024),
buildIndexCh: make(chan UniqueID, 1024),
notifyIndexChan: make(chan UniqueID),
dataNodeCreator: defaultDataNodeCreatorFunc,
rootCoordClientCreator: defaultRootCoordCreatorFunc,
helper: defaultServerHelper(),
@ -277,6 +283,7 @@ func (s *Server) Init() error {
if err = s.initSession(); err != nil {
return err
}
s.initIndexNodeManager()
if err = s.initServiceDiscovery(); err != nil {
return err
@ -289,6 +296,7 @@ func (s *Server) Init() error {
s.initSegmentManager()
s.initGarbageCollection(storageCli)
s.initIndexBuilder(storageCli)
return nil
}
@ -356,12 +364,8 @@ func (s *Server) SetEtcdClient(client *clientv3.Client) {
s.etcdCli = client
}
func (s *Server) SetIndexCoord(indexCoord types.IndexCoord) {
s.indexCoord = indexCoord
}
func (s *Server) createCompactionHandler() {
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh, s.segReferManager)
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh)
}
func (s *Server) stopCompactionHandler() {
@ -369,7 +373,7 @@ func (s *Server) stopCompactionHandler() {
}
func (s *Server) createCompactionTrigger() {
s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.segReferManager, s.indexCoord, s.handler)
s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.handler)
}
func (s *Server) stopCompactionTrigger() {
@ -387,7 +391,7 @@ func (s *Server) newChunkManagerFactory() (storage.ChunkManager, error) {
}
func (s *Server) initGarbageCollection(cli storage.ChunkManager) {
s.garbageCollector = newGarbageCollector(s.meta, s.handler, s.segReferManager, s.indexCoord, GcOption{
s.garbageCollector = newGarbageCollector(s.meta, s.handler, GcOption{
cli: cli,
enabled: Params.DataCoordCfg.EnableGarbageCollection.GetAsBool(),
checkInterval: Params.DataCoordCfg.GCInterval.GetAsDuration(time.Second),
@ -419,30 +423,29 @@ func (s *Server) initServiceDiscovery() error {
// TODO implement rewatch logic
s.dnEventCh = s.session.WatchServicesWithVersionRange(typeutil.DataNodeRole, r, rev+1, nil)
//icSessions, icRevision, err := s.session.GetSessions(typeutil.IndexCoordRole)
//if err != nil {
// log.Error("DataCoord get IndexCoord session failed", zap.Error(err))
// return err
//}
//serverIDs := make([]UniqueID, 0, len(icSessions))
//for _, session := range icSessions {
// serverIDs = append(serverIDs, session.ServerID)
//}
//s.icEventCh = s.session.WatchServices(typeutil.IndexCoordRole, icRevision+1, nil)
qcSessions, qcRevision, err := s.session.GetSessions(typeutil.QueryCoordRole)
inSessions, inRevision, err := s.session.GetSessions(typeutil.IndexNodeRole)
if err != nil {
log.Error("DataCoord get QueryCoord session failed", zap.Error(err))
return err
}
serverIDs := make([]UniqueID, 0, len(qcSessions))
for _, session := range qcSessions {
serverIDs = append(serverIDs, session.ServerID)
if Params.DataCoordCfg.BindIndexNodeMode.GetAsBool() {
if err = s.indexNodeManager.AddNode(Params.DataCoordCfg.IndexNodeID.GetAsInt64(), Params.DataCoordCfg.IndexNodeAddress.GetValue()); err != nil {
log.Error("add indexNode fail", zap.Int64("ServerID", Params.DataCoordCfg.IndexNodeID.GetAsInt64()),
zap.String("address", Params.DataCoordCfg.IndexNodeAddress.GetValue()), zap.Error(err))
return err
}
log.Info("add indexNode success", zap.String("IndexNode address", Params.DataCoordCfg.IndexNodeAddress.GetValue()),
zap.Int64("nodeID", Params.DataCoordCfg.IndexNodeID.GetAsInt64()))
} else {
for _, session := range inSessions {
if err := s.indexNodeManager.AddNode(session.ServerID, session.Address); err != nil {
return err
}
}
}
s.qcEventCh = s.session.WatchServices(typeutil.QueryCoordRole, qcRevision+1, nil)
s.inEventCh = s.session.WatchServices(typeutil.IndexNodeRole, inRevision+1, nil)
s.segReferManager, err = NewSegmentReferenceManager(s.kvClient, serverIDs)
return err
return nil
}
func (s *Server) initSegmentManager() {
@ -466,12 +469,25 @@ func (s *Server) initMeta(chunkManagerRootPath string, chunkManager storage.Chun
return retry.Do(s.ctx, reloadEtcdFn, retry.Attempts(connEtcdMaxRetryTime))
}
func (s *Server) initIndexBuilder(manager storage.ChunkManager) {
if s.indexBuilder == nil {
s.indexBuilder = newIndexBuilder(s.ctx, s.meta, s.indexNodeManager, manager)
}
}
func (s *Server) initIndexNodeManager() {
if s.indexNodeManager == nil {
s.indexNodeManager = NewNodeManager(s.ctx)
}
}
func (s *Server) startServerLoop() {
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx)
s.serverLoopWg.Add(3)
s.startDataNodeTtLoop(s.serverLoopCtx)
s.startWatchService(s.serverLoopCtx)
s.startFlushLoop(s.serverLoopCtx)
s.startIndexService(s.serverLoopCtx)
s.garbageCollector.start()
}
@ -644,25 +660,6 @@ func (s *Server) stopServiceWatch() {
}
}
func (s *Server) processSessionEvent(ctx context.Context, role string, event *sessionutil.SessionEvent) {
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("there is a new service online",
zap.String("server role", role),
zap.Int64("server ID", event.Session.ServerID))
case sessionutil.SessionDelEvent:
log.Warn("there is service offline",
zap.String("server role", role),
zap.Int64("server ID", event.Session.ServerID))
if err := retry.Do(ctx, func() error {
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
}, retry.Attempts(100)); err != nil {
panic(err)
}
}
}
// watchService watches services.
func (s *Server) watchService(ctx context.Context) {
defer logutil.LogPanic()
@ -677,7 +674,7 @@ func (s *Server) watchService(ctx context.Context) {
s.stopServiceWatch()
return
}
if err := s.handleSessionEvent(ctx, event); err != nil {
if err := s.handleSessionEvent(ctx, typeutil.DataNodeRole, event); err != nil {
go func() {
if err := s.Stop(); err != nil {
log.Warn("DataCoord server stop error", zap.Error(err))
@ -685,59 +682,84 @@ func (s *Server) watchService(ctx context.Context) {
}()
return
}
//case event, ok := <-s.icEventCh:
// if !ok {
// s.stopServiceWatch()
// return
// }
// s.processSessionEvent(ctx, "IndexCoord", event)
case event, ok := <-s.qcEventCh:
case event, ok := <-s.inEventCh:
if !ok {
s.stopServiceWatch()
return
}
s.processSessionEvent(ctx, "QueryCoord", event)
if err := s.handleSessionEvent(ctx, typeutil.IndexNodeRole, event); err != nil {
go func() {
if err := s.Stop(); err != nil {
log.Warn("DataCoord server stop error", zap.Error(err))
}
}()
return
}
}
}
}
// handles session events - DataNodes Add/Del
func (s *Server) handleSessionEvent(ctx context.Context, event *sessionutil.SessionEvent) error {
func (s *Server) handleSessionEvent(ctx context.Context, role string, event *sessionutil.SessionEvent) error {
if event == nil {
return nil
}
info := &datapb.DataNodeInfo{
Address: event.Session.Address,
Version: event.Session.ServerID,
Channels: []*datapb.ChannelStatus{},
}
node := &NodeInfo{
NodeID: event.Session.ServerID,
Address: event.Session.Address,
}
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("received datanode register",
zap.String("address", info.Address),
zap.Int64("serverID", info.Version))
if err := s.cluster.Register(node); err != nil {
log.Warn("failed to register node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
return err
switch role {
case typeutil.DataNodeRole:
info := &datapb.DataNodeInfo{
Address: event.Session.Address,
Version: event.Session.ServerID,
Channels: []*datapb.ChannelStatus{},
}
s.metricsCacheManager.InvalidateSystemInfoMetrics()
case sessionutil.SessionDelEvent:
log.Info("received datanode unregister",
zap.String("address", info.Address),
zap.Int64("serverID", info.Version))
if err := s.cluster.UnRegister(node); err != nil {
log.Warn("failed to deregister node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
return err
node := &NodeInfo{
NodeID: event.Session.ServerID,
Address: event.Session.Address,
}
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("received datanode register",
zap.String("address", info.Address),
zap.Int64("serverID", info.Version))
if err := s.cluster.Register(node); err != nil {
log.Warn("failed to register node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
return err
}
s.metricsCacheManager.InvalidateSystemInfoMetrics()
case sessionutil.SessionDelEvent:
log.Info("received datanode unregister",
zap.String("address", info.Address),
zap.Int64("serverID", info.Version))
if err := s.cluster.UnRegister(node); err != nil {
log.Warn("failed to deregister node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
return err
}
s.metricsCacheManager.InvalidateSystemInfoMetrics()
default:
log.Warn("receive unknown service event type",
zap.Any("type", event.EventType))
}
case typeutil.IndexNodeRole:
switch event.EventType {
case sessionutil.SessionAddEvent:
log.Info("received indexnode register",
zap.String("address", event.Session.Address),
zap.Int64("serverID", event.Session.ServerID))
return s.indexNodeManager.AddNode(event.Session.ServerID, event.Session.Address)
case sessionutil.SessionDelEvent:
log.Info("received indexnode unregister",
zap.String("address", event.Session.Address),
zap.Int64("serverID", event.Session.ServerID))
s.indexNodeManager.RemoveNode(event.Session.ServerID)
case sessionutil.SessionUpdateEvent:
serverID := event.Session.ServerID
log.Info("received indexnode SessionUpdateEvent", zap.Int64("serverID", serverID))
s.indexNodeManager.StoppingNode(serverID)
default:
log.Warn("receive unknown service event type",
zap.Any("type", event.EventType))
}
s.metricsCacheManager.InvalidateSystemInfoMetrics()
default:
log.Warn("receive unknown service event type",
zap.Any("type", event.EventType))
}
return nil
}
@ -782,6 +804,7 @@ func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
log.Error("flush segment complete failed", zap.Error(err))
return err
}
s.buildIndexCh <- segmentID
log.Info("flush segment complete", zap.Int64("id", segmentID))
return nil
}

View File

@ -30,13 +30,6 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
@ -47,8 +40,8 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/common"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
@ -58,8 +51,12 @@ import (
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
func TestMain(m *testing.M) {
@ -989,73 +986,70 @@ func TestService_WatchServices(t *testing.T) {
// assert.True(t, closed)
//}
func TestServer_watchQueryCoord(t *testing.T) {
Params.Init()
etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
Params.EtcdCfg.Endpoints.GetAsStrings(),
Params.EtcdCfg.EtcdTLSCert.GetValue(),
Params.EtcdCfg.EtcdTLSKey.GetValue(),
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.Nil(t, err)
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
assert.NotNil(t, etcdKV)
factory := dependency.NewDefaultFactory(true)
svr := CreateServer(context.TODO(), factory)
svr.session = &sessionutil.Session{
TriggerKill: true,
}
svr.kvClient = etcdKV
dnCh := make(chan *sessionutil.SessionEvent)
//icCh := make(chan *sessionutil.SessionEvent)
qcCh := make(chan *sessionutil.SessionEvent)
svr.dnEventCh = dnCh
//svr.icEventCh = icCh
svr.qcEventCh = qcCh
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
assert.NoError(t, err)
assert.NotNil(t, segRefer)
svr.segReferManager = segRefer
sc := make(chan os.Signal, 1)
signal.Notify(sc, syscall.SIGINT)
defer signal.Reset(syscall.SIGINT)
closed := false
sigQuit := make(chan struct{}, 1)
svr.serverLoopWg.Add(1)
go func() {
svr.watchService(context.Background())
}()
go func() {
<-sc
closed = true
sigQuit <- struct{}{}
}()
qcCh <- &sessionutil.SessionEvent{
EventType: sessionutil.SessionAddEvent,
Session: &sessionutil.Session{
ServerID: 2,
},
}
qcCh <- &sessionutil.SessionEvent{
EventType: sessionutil.SessionDelEvent,
Session: &sessionutil.Session{
ServerID: 2,
},
}
close(qcCh)
<-sigQuit
svr.serverLoopWg.Wait()
assert.True(t, closed)
}
//func TestServer_watchQueryCoord(t *testing.T) {
// Params.Init()
// etcdCli, err := etcd.GetEtcdClient(
// Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
// Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
// Params.EtcdCfg.Endpoints.GetAsStrings(),
// Params.EtcdCfg.EtcdTLSCert.GetValue(),
// Params.EtcdCfg.EtcdTLSKey.GetValue(),
// Params.EtcdCfg.EtcdTLSCACert.GetValue(),
// Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
// assert.Nil(t, err)
// etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
// assert.NotNil(t, etcdKV)
// factory := dependency.NewDefaultFactory(true)
// svr := CreateServer(context.TODO(), factory)
// svr.session = &sessionutil.Session{
// TriggerKill: true,
// }
// svr.kvClient = etcdKV
//
// dnCh := make(chan *sessionutil.SessionEvent)
// //icCh := make(chan *sessionutil.SessionEvent)
// qcCh := make(chan *sessionutil.SessionEvent)
//
// svr.dnEventCh = dnCh
//
// segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
// assert.NoError(t, err)
// assert.NotNil(t, segRefer)
//
// sc := make(chan os.Signal, 1)
// signal.Notify(sc, syscall.SIGINT)
// defer signal.Reset(syscall.SIGINT)
// closed := false
// sigQuit := make(chan struct{}, 1)
//
// svr.serverLoopWg.Add(1)
// go func() {
// svr.watchService(context.Background())
// }()
//
// go func() {
// <-sc
// closed = true
// sigQuit <- struct{}{}
// }()
//
// qcCh <- &sessionutil.SessionEvent{
// EventType: sessionutil.SessionAddEvent,
// Session: &sessionutil.Session{
// ServerID: 2,
// },
// }
// qcCh <- &sessionutil.SessionEvent{
// EventType: sessionutil.SessionDelEvent,
// Session: &sessionutil.Session{
// ServerID: 2,
// },
// }
// close(qcCh)
// <-sigQuit
// svr.serverLoopWg.Wait()
// assert.True(t, closed)
//}
func TestServer_ShowConfigurations(t *testing.T) {
svr := newTestServer(t, nil)
@ -1961,6 +1955,14 @@ func TestGetQueryVChanPositions(t *testing.T) {
},
})
err := svr.meta.CreateIndex(&model.Index{
TenantID: "",
CollectionID: 0,
FieldID: 2,
IndexID: 1,
})
assert.Nil(t, err)
s1 := &datapb.SegmentInfo{
ID: 1,
CollectionID: 0,
@ -1974,7 +1976,18 @@ func TestGetQueryVChanPositions(t *testing.T) {
Timestamp: 0,
},
}
err := svr.meta.AddSegment(NewSegmentInfo(s1))
err = svr.meta.AddSegment(NewSegmentInfo(s1))
assert.Nil(t, err)
err = svr.meta.AddSegmentIndex(&model.SegmentIndex{
SegmentID: 1,
BuildID: 1,
IndexID: 1,
})
assert.Nil(t, err)
err = svr.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: 1,
State: commonpb.IndexState_Finished,
})
assert.Nil(t, err)
s2 := &datapb.SegmentInfo{
ID: 2,
@ -2016,24 +2029,22 @@ func TestGetQueryVChanPositions(t *testing.T) {
}
err = svr.meta.AddSegment(NewSegmentInfo(s3))
assert.Nil(t, err)
mockResp := &indexpb.GetIndexInfoResponse{
Status: &commonpb.Status{},
SegmentInfo: map[int64]*indexpb.SegmentInfo{
s1.ID: {
CollectionID: s1.CollectionID,
SegmentID: s1.ID,
EnableIndex: true,
IndexInfos: []*indexpb.IndexFilePathInfo{
{
SegmentID: s1.ID,
FieldID: 2,
},
},
},
},
}
svr.indexCoord = mocks.NewMockIndexCoord(t)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(mockResp, nil)
//mockResp := &indexpb.GetIndexInfoResponse{
// Status: &commonpb.Status{},
// SegmentInfo: map[int64]*indexpb.SegmentInfo{
// s1.ID: {
// CollectionID: s1.CollectionID,
// SegmentID: s1.ID,
// EnableIndex: true,
// IndexInfos: []*indexpb.IndexFilePathInfo{
// {
// SegmentID: s1.ID,
// FieldID: 2,
// },
// },
// },
// },
//}
t.Run("get unexisted channel", func(t *testing.T) {
vchan := svr.handler.GetQueryVChanPositions(&channel{Name: "chx1", CollectionID: 0}, allPartitionID)
@ -2074,17 +2085,6 @@ func TestGetQueryVChanPositions(t *testing.T) {
assert.EqualValues(t, 0, infos.CollectionID)
assert.EqualValues(t, vchannel, infos.ChannelName)
})
t.Run("filter non indexed segments", func(t *testing.T) {
svr.indexCoord = mocks.NewMockIndexCoord(t)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(
&indexpb.GetIndexInfoResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}}, nil)
vchan := svr.handler.GetQueryVChanPositions(&channel{Name: "ch1", CollectionID: 0}, allPartitionID)
assert.EqualValues(t, 0, len(vchan.FlushedSegmentIds))
assert.EqualValues(t, 3, len(vchan.UnflushedSegmentIds))
assert.ElementsMatch(t, []int64{s1.ID, s2.ID, s3.ID}, vchan.UnflushedSegmentIds)
})
}
func TestShouldDropChannel(t *testing.T) {
@ -2297,6 +2297,15 @@ func TestGetRecoveryInfo(t *testing.T) {
})
assert.NoError(t, err)
err = svr.meta.CreateIndex(&model.Index{
TenantID: "",
CollectionID: 0,
FieldID: 2,
IndexID: 0,
IndexName: "",
})
assert.Nil(t, err)
seg1 := createSegment(0, 0, 0, 100, 10, "vchan1", commonpb.SegmentState_Flushed)
seg1.Binlogs = []*datapb.FieldBinlog{
{
@ -2337,35 +2346,26 @@ func TestGetRecoveryInfo(t *testing.T) {
assert.Nil(t, err)
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
assert.Nil(t, err)
mockResp := &indexpb.GetIndexInfoResponse{
Status: &commonpb.Status{},
SegmentInfo: map[int64]*indexpb.SegmentInfo{
seg1.ID: {
CollectionID: seg1.CollectionID,
SegmentID: seg1.ID,
EnableIndex: true,
IndexInfos: []*indexpb.IndexFilePathInfo{
{
SegmentID: seg1.ID,
FieldID: 2,
},
},
},
seg2.ID: {
CollectionID: seg2.CollectionID,
SegmentID: seg2.ID,
EnableIndex: true,
IndexInfos: []*indexpb.IndexFilePathInfo{
{
SegmentID: seg2.ID,
FieldID: 2,
},
},
},
},
}
svr.indexCoord = mocks.NewMockIndexCoord(t)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(mockResp, nil)
err = svr.meta.AddSegmentIndex(&model.SegmentIndex{
SegmentID: seg1.ID,
BuildID: seg1.ID,
})
assert.Nil(t, err)
err = svr.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: seg1.ID,
State: commonpb.IndexState_Finished,
})
assert.Nil(t, err)
err = svr.meta.AddSegmentIndex(&model.SegmentIndex{
SegmentID: seg2.ID,
BuildID: seg2.ID,
})
assert.Nil(t, err)
err = svr.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: seg2.ID,
State: commonpb.IndexState_Finished,
})
assert.Nil(t, err)
req := &datapb.GetRecoveryInfoRequest{
CollectionID: 0,
@ -2442,7 +2442,7 @@ func TestGetRecoveryInfo(t *testing.T) {
assert.Nil(t, err)
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
assert.Nil(t, err)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil)
//svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil)
req := &datapb.GetRecoveryInfoRequest{
CollectionID: 0,
@ -2515,24 +2515,25 @@ func TestGetRecoveryInfo(t *testing.T) {
err := svr.meta.AddSegment(NewSegmentInfo(segment))
assert.Nil(t, err)
mockResp := &indexpb.GetIndexInfoResponse{
Status: &commonpb.Status{},
SegmentInfo: map[int64]*indexpb.SegmentInfo{
segment.ID: {
CollectionID: segment.CollectionID,
SegmentID: segment.ID,
EnableIndex: true,
IndexInfos: []*indexpb.IndexFilePathInfo{
{
SegmentID: segment.ID,
FieldID: 2,
},
},
},
},
}
svr.indexCoord = mocks.NewMockIndexCoord(t)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(mockResp, nil)
err = svr.meta.CreateIndex(&model.Index{
TenantID: "",
CollectionID: 0,
FieldID: 2,
IndexID: 0,
IndexName: "",
})
assert.Nil(t, err)
err = svr.meta.AddSegmentIndex(&model.SegmentIndex{
SegmentID: segment.ID,
BuildID: segment.ID,
})
assert.Nil(t, err)
err = svr.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: segment.ID,
State: commonpb.IndexState_Finished,
})
assert.Nil(t, err)
err = svr.channelManager.AddNode(0)
assert.Nil(t, err)
err = svr.channelManager.Watch(&channel{Name: "vchan1", CollectionID: 0})
@ -2582,7 +2583,6 @@ func TestGetRecoveryInfo(t *testing.T) {
assert.Nil(t, err)
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
assert.Nil(t, err)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil)
req := &datapb.GetRecoveryInfoRequest{
CollectionID: 0,
@ -2625,7 +2625,6 @@ func TestGetRecoveryInfo(t *testing.T) {
assert.Nil(t, err)
err = svr.meta.AddSegment(NewSegmentInfo(seg2))
assert.Nil(t, err)
svr.indexCoord.(*mocks.MockIndexCoord).EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil)
req := &datapb.GetRecoveryInfoRequest{
CollectionID: 0,
@ -2932,7 +2931,7 @@ func TestHandleSessionEvent(t *testing.T) {
Exclusive: false,
},
}
err = svr.handleSessionEvent(context.Background(), evt)
err = svr.handleSessionEvent(context.Background(), typeutil.DataNodeRole, evt)
assert.Nil(t, err)
evt = &sessionutil.SessionEvent{
@ -2944,7 +2943,7 @@ func TestHandleSessionEvent(t *testing.T) {
Exclusive: false,
},
}
err = svr.handleSessionEvent(context.Background(), evt)
err = svr.handleSessionEvent(context.Background(), typeutil.DataNodeRole, evt)
assert.Nil(t, err)
dataNodes := svr.cluster.GetSessions()
assert.EqualValues(t, 1, len(dataNodes))
@ -2959,7 +2958,7 @@ func TestHandleSessionEvent(t *testing.T) {
Exclusive: false,
},
}
err = svr.handleSessionEvent(context.Background(), evt)
err = svr.handleSessionEvent(context.Background(), typeutil.DataNodeRole, evt)
assert.Nil(t, err)
dataNodes = svr.cluster.GetSessions()
assert.EqualValues(t, 0, len(dataNodes))
@ -2967,7 +2966,7 @@ func TestHandleSessionEvent(t *testing.T) {
t.Run("nil evt", func(t *testing.T) {
assert.NotPanics(t, func() {
err = svr.handleSessionEvent(context.Background(), nil)
err = svr.handleSessionEvent(context.Background(), typeutil.DataNodeRole, nil)
assert.Nil(t, err)
})
})
@ -3235,7 +3234,6 @@ func TestDataCoord_Import(t *testing.T) {
t.Run("no datanode available", func(t *testing.T) {
svr := newTestServer(t, nil)
Params.BaseTable.Save("minio.address", "minio:9000")
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
ImportTask: &datapb.ImportTask{
CollectionId: 100,
@ -3247,6 +3245,9 @@ func TestDataCoord_Import(t *testing.T) {
closeTestServer(t, svr)
})
// just passed for ci, if test locally, need to replace it with localhost:9000
Params.BaseTable.Save("minio.address", "minio:9000")
t.Run("with closed server", func(t *testing.T) {
svr := newTestServer(t, nil)
closeTestServer(t, svr)
@ -3290,29 +3291,6 @@ func TestDataCoord_Import(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
})
t.Run("test acquire segment reference lock with closed server", func(t *testing.T) {
svr := newTestServer(t, nil)
closeTestServer(t, svr)
status, err := svr.AcquireSegmentLock(context.TODO(), &datapb.AcquireSegmentLockRequest{
SegmentIDs: []UniqueID{1, 2},
NodeID: UniqueID(1),
})
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
})
t.Run("test release segment reference lock with closed server", func(t *testing.T) {
svr := newTestServer(t, nil)
closeTestServer(t, svr)
status, err := svr.ReleaseSegmentLock(context.TODO(), &datapb.ReleaseSegmentLockRequest{
TaskID: UniqueID(1),
NodeID: UniqueID(1),
})
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
})
}
func TestDataCoord_SaveImportSegment(t *testing.T) {
@ -3534,9 +3512,6 @@ func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
}
indexCoord := mocks.NewMockIndexCoord(t)
indexCoord.EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
svr.indexCoord = indexCoord
err = svr.Init()
assert.Nil(t, err)
@ -3579,9 +3554,9 @@ func newTestServerWithMeta(t *testing.T, receiveCh chan any, meta *meta, opts ..
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
}
indexCoord := mocks.NewMockIndexCoord(t)
indexCoord.EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
svr.indexCoord = indexCoord
//indexCoord := mocks.NewMockIndexCoord(t)
//indexCoord.EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
//svr.indexCoord = indexCoord
err = svr.Init()
assert.Nil(t, err)
@ -3723,45 +3698,45 @@ func Test_CheckHealth(t *testing.T) {
})
}
func Test_initServiceDiscovery(t *testing.T) {
server := newTestServer2(t, nil)
assert.NotNil(t, server)
segmentID := rand.Int63()
err := server.meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: segmentID,
CollectionID: rand.Int63(),
PartitionID: rand.Int63(),
NumOfRows: 100,
},
currRows: 100,
})
assert.Nil(t, err)
qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), server.etcdCli)
qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true)
qcSession.Register()
req := &datapb.AcquireSegmentLockRequest{
NodeID: qcSession.ServerID,
SegmentIDs: []UniqueID{segmentID},
}
resp, err := server.AcquireSegmentLock(context.TODO(), req)
assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.QueryCoordRole)
_, err = server.etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err)
for {
if !server.segReferManager.HasSegmentLock(segmentID) {
break
}
}
closeTestServer(t, server)
}
//func Test_initServiceDiscovery(t *testing.T) {
// server := newTestServer2(t, nil)
// assert.NotNil(t, server)
//
// segmentID := rand.Int63()
// err := server.meta.AddSegment(&SegmentInfo{
// SegmentInfo: &datapb.SegmentInfo{
// ID: segmentID,
// CollectionID: rand.Int63(),
// PartitionID: rand.Int63(),
// NumOfRows: 100,
// },
// currRows: 100,
// })
// assert.Nil(t, err)
//
// qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), server.etcdCli)
// qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true)
// qcSession.Register()
// //req := &datapb.AcquireSegmentLockRequest{
// // NodeID: qcSession.ServerID,
// // SegmentIDs: []UniqueID{segmentID},
// //}
// //resp, err := server.AcquireSegmentLock(context.TODO(), req)
// //assert.Nil(t, err)
// //assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
//
// sessKey := path.Join(Params.EtcdCfg.MetaRootPath.GetValue(), sessionutil.DefaultServiceRoot, typeutil.QueryCoordRole)
// _, err = server.etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
// assert.Nil(t, err)
//
// //for {
// // if !server.segReferManager.HasSegmentLock(segmentID) {
// // break
// // }
// //}
//
// closeTestServer(t, server)
//}
func Test_newChunkManagerFactory(t *testing.T) {
server := newTestServer2(t, nil)

View File

@ -23,7 +23,9 @@ import (
"strconv"
"sync"
"github.com/milvus-io/milvus/internal/util/segmentutil"
"github.com/samber/lo"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
@ -36,13 +38,10 @@ import (
"github.com/milvus-io/milvus/internal/util/logutil"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/segmentutil"
"github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/samber/lo"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
// checks whether server in Healthy State
@ -1245,69 +1244,6 @@ func getDiff(base, remove []int64) []int64 {
return diff
}
// AcquireSegmentLock acquire the reference lock of the segments.
func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
resp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
}
if s.isClosed() {
log.Warn("failed to acquire segments reference lock for closed server")
resp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return resp, nil
}
hasSegments, err := s.meta.HasSegments(req.SegmentIDs)
if !hasSegments || err != nil {
log.Error("AcquireSegmentLock failed", zap.Error(err))
resp.Reason = err.Error()
return resp, nil
}
err = s.segReferManager.AddSegmentsLock(req.TaskID, req.SegmentIDs, req.NodeID)
if err != nil {
log.Warn("Add reference lock on segments failed", zap.Int64s("segIDs", req.SegmentIDs), zap.Error(err))
resp.Reason = err.Error()
return resp, nil
}
hasSegments, err = s.meta.HasSegments(req.SegmentIDs)
if !hasSegments || err != nil {
log.Error("AcquireSegmentLock failed, try to release reference lock", zap.Error(err))
if err2 := retry.Do(ctx, func() error {
return s.segReferManager.ReleaseSegmentsLock(req.TaskID, req.NodeID)
}, retry.Attempts(100)); err2 != nil {
panic(err)
}
resp.Reason = err.Error()
return resp, nil
}
resp.ErrorCode = commonpb.ErrorCode_Success
return resp, nil
}
// ReleaseSegmentLock release the reference lock of the segments.
func (s *Server) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
resp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
}
if s.isClosed() {
log.Warn("failed to release segments reference lock for closed server")
resp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return resp, nil
}
err := s.segReferManager.ReleaseSegmentsLock(req.TaskID, req.NodeID)
if err != nil {
log.Error("DataCoord ReleaseSegmentLock failed", zap.Int64("taskID", req.TaskID), zap.Int64("nodeID", req.NodeID),
zap.Error(err))
resp.Reason = err.Error()
return resp, nil
}
resp.ErrorCode = commonpb.ErrorCode_Success
return resp, nil
}
// SaveImportSegment saves the segment binlog paths and puts this segment to its belonging DataNode as a flushed segment.
func (s *Server) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
log.Info("DataCoord putting segment to the right DataNode and saving binlog path",

View File

@ -4,10 +4,10 @@ import (
"context"
"testing"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
func TestBroadcastAlteredCollection(t *testing.T) {

View File

@ -20,18 +20,16 @@ import (
"context"
"errors"
"strconv"
"sync"
"strings"
"time"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap"
)
// Response response interface for verification
@ -101,7 +99,7 @@ func GetCompactTime(ctx context.Context, allocator allocator) (*compactTime, err
return &compactTime{ttRetentionLogic, 0, 0}, nil
}
func FilterInIndexedSegments(handler Handler, indexCoord types.IndexCoord, segments ...*SegmentInfo) []*SegmentInfo {
func FilterInIndexedSegments(handler Handler, mt *meta, segments ...*SegmentInfo) []*SegmentInfo {
if len(segments) == 0 {
return nil
}
@ -133,64 +131,17 @@ func FilterInIndexedSegments(handler Handler, indexCoord types.IndexCoord, segme
}
}
wg := sync.WaitGroup{}
indexedSegmentCh := make(chan []int64, len(segments))
for _, segment := range segments {
segment := segment
wg.Add(1)
go func() {
defer wg.Done()
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
resp, err := indexCoord.GetIndexInfos(ctx, &indexpb.GetIndexInfoRequest{
CollectionID: segment.GetCollectionID(),
SegmentIDs: []int64{segment.GetID()},
})
if err != nil || resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("failed to get index of collection",
zap.Int64("collectionID", segment.GetCollectionID()),
zap.Int64("segmentID", segment.GetID()))
return
}
indexed := extractSegmentsWithVectorIndex(vecFieldID, resp.GetSegmentInfo())
if len(indexed) == 0 {
log.Info("no vector index for the segment",
zap.Int64("collectionID", segment.GetCollectionID()),
zap.Int64("segmentID", segment.GetID()))
return
}
indexedSegmentCh <- indexed
}()
}
wg.Wait()
close(indexedSegmentCh)
indexedSegments := make([]*SegmentInfo, 0)
for segments := range indexedSegmentCh {
for _, segment := range segments {
if info, ok := segmentMap[segment]; ok {
delete(segmentMap, segment)
indexedSegments = append(indexedSegments, info)
}
for _, segment := range segments {
segmentState := mt.GetSegmentIndexStateOnField(segment.GetCollectionID(), segment.GetID(), vecFieldID[segment.GetCollectionID()])
if segmentState.state == commonpb.IndexState_Finished {
indexedSegments = append(indexedSegments, segment)
}
}
return indexedSegments
}
func extractSegmentsWithVectorIndex(vecFieldID map[int64]int64, segentIndexInfo map[int64]*indexpb.SegmentInfo) []int64 {
indexedSegments := make(typeutil.UniqueSet)
for _, indexInfo := range segentIndexInfo {
for _, index := range indexInfo.GetIndexInfos() {
if index.GetFieldID() == vecFieldID[indexInfo.GetCollectionID()] {
indexedSegments.Insert(indexInfo.GetSegmentID())
break
}
}
}
return indexedSegments.Collect()
}
func getZeroTime() time.Time {
var t time.Time
return t
@ -209,3 +160,24 @@ func getCollectionTTL(properties map[string]string) (time.Duration, error) {
return Params.CommonCfg.EntityExpirationTTL.GetAsDuration(time.Second), nil
}
func getIndexType(indexParams []*commonpb.KeyValuePair) string {
for _, param := range indexParams {
if param.Key == "index_type" {
return param.Value
}
}
return invalidIndex
}
func isFlatIndex(indexType string) bool {
return indexType == flatIndex || indexType == binFlatIndex
}
func parseBuildIDFromFilePath(key string) (UniqueID, error) {
ss := strings.Split(key, "/")
if strings.HasSuffix(key, "/") {
return strconv.ParseInt(ss[len(ss)-2], 10, 64)
}
return strconv.ParseInt(ss[len(ss)-1], 10, 64)
}

View File

@ -54,8 +54,6 @@ type ConnectionManager struct {
queryCoordMu sync.RWMutex
dataCoord datapb.DataCoordClient
dataCoordMu sync.RWMutex
indexCoord indexpb.IndexCoordClient
indexCoordMu sync.RWMutex
queryNodes map[int64]querypb.QueryNodeClient
queryNodesMu sync.RWMutex
dataNodes map[int64]datapb.DataNodeClient
@ -163,18 +161,6 @@ func (cm *ConnectionManager) GetDataCoordClient() (datapb.DataCoordClient, bool)
return cm.dataCoord, true
}
func (cm *ConnectionManager) GetIndexCoordClient() (indexpb.IndexCoordClient, bool) {
cm.indexCoordMu.RLock()
defer cm.indexCoordMu.RUnlock()
_, ok := cm.dependencies[typeutil.IndexCoordRole]
if !ok {
log.Error("IndeCoord dependency has not been added yet")
return nil, false
}
return cm.indexCoord, true
}
func (cm *ConnectionManager) GetQueryNodeClients() (map[int64]querypb.QueryNodeClient, bool) {
cm.queryNodesMu.RLock()
defer cm.queryNodesMu.RUnlock()
@ -291,10 +277,6 @@ func (cm *ConnectionManager) buildClients(session *sessionutil.Session, connecti
cm.dataCoordMu.Lock()
defer cm.dataCoordMu.Unlock()
cm.dataCoord = datapb.NewDataCoordClient(connection)
case typeutil.IndexCoordRole:
cm.indexCoordMu.Lock()
defer cm.indexCoordMu.Unlock()
cm.indexCoord = indexpb.NewIndexCoordClient(connection)
case typeutil.QueryCoordRole:
cm.queryCoordMu.Lock()
defer cm.queryCoordMu.Unlock()

View File

@ -106,23 +106,6 @@ func TestConnectionManager(t *testing.T) {
}, 10*time.Second, 100*time.Millisecond)
})
t.Run("indexCoord", func(t *testing.T) {
lis, err := net.Listen("tcp", "127.0.0.1:")
assert.Nil(t, err)
defer lis.Close()
indexCoord := &testIndexCoord{}
grpcServer := grpc.NewServer()
defer grpcServer.Stop()
indexpb.RegisterIndexCoordServer(grpcServer, indexCoord)
go grpcServer.Serve(lis)
session.Init(typeutil.IndexCoordRole, lis.Addr().String(), true, false)
session.Register()
assert.Eventually(t, func() bool {
indexCoord, ok := cm.GetIndexCoordClient()
return indexCoord != nil && ok
}, 10*time.Second, 100*time.Millisecond)
})
t.Run("queryNode", func(t *testing.T) {
lis, err := net.Listen("tcp", "127.0.0.1:")
assert.Nil(t, err)
@ -251,10 +234,6 @@ type testDataCoord struct {
datapb.DataCoordServer
}
type testIndexCoord struct {
indexpb.IndexCoordServer
}
type testQueryNode struct {
querypb.QueryNodeServer
}

View File

@ -21,6 +21,10 @@ import (
"fmt"
"time"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
@ -33,9 +37,6 @@ import (
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
)
var Params *paramtable.ComponentParam = paramtable.Get()
@ -675,44 +676,6 @@ func (c *Client) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update
return ret.(*commonpb.Status), err
}
// AcquireSegmentLock acquire the reference lock of the segments.
func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.sess.ServerID)),
)
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.AcquireSegmentLock(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}
// ReleaseSegmentLock release the reference lock of the segments.
func (c *Client) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.sess.ServerID)),
)
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.ReleaseSegmentLock(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}
// SaveImportSegment is the DataCoord client side code for SaveImportSegment call.
func (c *Client) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
@ -795,3 +758,101 @@ func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthReque
}
return ret.(*milvuspb.CheckHealthResponse), err
}
// CreateIndex sends the build index request to IndexCoord.
func (c *Client) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.CreateIndex(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}
// GetIndexState gets the index states from IndexCoord.
func (c *Client) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetIndexState(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*datapb.GetIndexStateResponse), err
}
// GetSegmentIndexState gets the index states from IndexCoord.
func (c *Client) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetSegmentIndexState(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*datapb.GetSegmentIndexStateResponse), err
}
// GetIndexInfos gets the index file paths from IndexCoord.
func (c *Client) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetIndexInfos(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*datapb.GetIndexInfoResponse), err
}
// DescribeIndex describe the index info of the collection.
func (c *Client) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.DescribeIndex(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*datapb.DescribeIndexResponse), err
}
// GetIndexBuildProgress describe the progress of the index.
func (c *Client) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetIndexBuildProgress(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*datapb.GetIndexBuildProgressResponse), err
}
// DropIndex sends the drop index request to IndexCoord.
func (c *Client) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.DropIndex(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}

View File

@ -135,12 +135,6 @@ func Test_NewClient(t *testing.T) {
r24, err := client.UpdateSegmentStatistics(ctx, nil)
retCheck(retNotNil, r24, err)
r25, err := client.AcquireSegmentLock(ctx, nil)
retCheck(retNotNil, r25, err)
r26, err := client.ReleaseSegmentLock(ctx, nil)
retCheck(retNotNil, r26, err)
r27, err := client.SaveImportSegment(ctx, nil)
retCheck(retNotNil, r27, err)
@ -153,6 +147,30 @@ func Test_NewClient(t *testing.T) {
r31, err := client.ShowConfigurations(ctx, nil)
retCheck(retNotNil, r31, err)
r32, err := client.CreateIndex(ctx, nil)
retCheck(retNotNil, r32, err)
r33, err := client.DescribeIndex(ctx, nil)
retCheck(retNotNil, r33, err)
r34, err := client.DropIndex(ctx, nil)
retCheck(retNotNil, r34, err)
r35, err := client.GetIndexState(ctx, nil)
retCheck(retNotNil, r35, err)
r36, err := client.GetIndexBuildProgress(ctx, nil)
retCheck(retNotNil, r36, err)
r37, err := client.GetIndexInfos(ctx, nil)
retCheck(retNotNil, r37, err)
r38, err := client.GetSegmentIndexState(ctx, nil)
retCheck(retNotNil, r38, err)
r39, err := client.UpdateChannelCheckpoint(ctx, nil)
retCheck(retNotNil, r39, err)
{
ret, err := client.BroadcastAlteredCollection(ctx, nil)
retCheck(retNotNil, ret, err)

View File

@ -27,7 +27,6 @@ import (
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
@ -56,8 +55,7 @@ type Server struct {
wg sync.WaitGroup
dataCoord types.DataCoordComponent
etcdCli *clientv3.Client
indexCoord types.IndexCoord
etcdCli *clientv3.Client
grpcErrChan chan error
grpcServer *grpc.Server
@ -100,25 +98,6 @@ func (s *Server) init() error {
s.dataCoord.SetEtcdClient(etcdCli)
s.dataCoord.SetAddress(Params.GetAddress())
if s.indexCoord == nil {
var err error
log.Debug("create IndexCoord client for DataCoord")
s.indexCoord, err = icc.NewClient(s.ctx, etcdConfig.MetaRootPath.GetValue(), etcdCli)
if err != nil {
log.Warn("failed to create IndexCoord client for DataCoord", zap.Error(err))
return err
}
log.Debug("create IndexCoord client for DataCoord done")
}
log.Debug("init IndexCoord client for DataCoord")
if err := s.indexCoord.Init(); err != nil {
log.Warn("failed to init IndexCoord client for DataCoord", zap.Error(err))
return err
}
log.Debug("init IndexCoord client for DataCoord done")
s.dataCoord.SetIndexCoord(s.indexCoord)
err = s.startGrpc()
if err != nil {
log.Debug("DataCoord startGrpc failed", zap.Error(err))
@ -380,16 +359,6 @@ func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update
return s.dataCoord.UpdateChannelCheckpoint(ctx, req)
}
// AcquireSegmentLock acquire the reference lock of the segments.
func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
return s.dataCoord.AcquireSegmentLock(ctx, req)
}
// ReleaseSegmentLock release the reference lock of the segments.
func (s *Server) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return s.dataCoord.ReleaseSegmentLock(ctx, req)
}
// SaveImportSegment saves the import segment binlog paths data and then looks for the right DataNode to add the
// segment to that DataNode.
func (s *Server) SaveImportSegment(ctx context.Context, request *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
@ -413,3 +382,38 @@ func (s *Server) BroadcastAlteredCollection(ctx context.Context, request *datapb
func (s *Server) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
return s.dataCoord.CheckHealth(ctx, req)
}
// CreateIndex sends the build index request to DataCoord.
func (s *Server) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
return s.dataCoord.CreateIndex(ctx, req)
}
// GetIndexState gets the index states from DataCoord.
// Deprecated: use DescribeIndex instead
func (s *Server) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
return s.dataCoord.GetIndexState(ctx, req)
}
func (s *Server) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return s.dataCoord.GetSegmentIndexState(ctx, req)
}
// GetIndexInfos gets the index file paths from DataCoord.
func (s *Server) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
return s.dataCoord.GetIndexInfos(ctx, req)
}
// DescribeIndex gets all indexes of the collection.
func (s *Server) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return s.dataCoord.DescribeIndex(ctx, req)
}
// DropIndex sends the drop index request to DataCoord.
func (s *Server) DropIndex(ctx context.Context, request *datapb.DropIndexRequest) (*commonpb.Status, error) {
return s.dataCoord.DropIndex(ctx, request)
}
// Deprecated: use DescribeIndex instead
func (s *Server) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
return s.dataCoord.GetIndexBuildProgress(ctx, req)
}

View File

@ -23,7 +23,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/types"
@ -64,12 +63,18 @@ type MockDataCoord struct {
importResp *datapb.ImportTaskResponse
updateSegStatResp *commonpb.Status
updateChanPos *commonpb.Status
acquireSegLockResp *commonpb.Status
releaseSegLockResp *commonpb.Status
addSegmentResp *commonpb.Status
unsetIsImportingStateResp *commonpb.Status
markSegmentsDroppedResp *commonpb.Status
broadCastResp *commonpb.Status
createIndexResp *commonpb.Status
describeIndexResp *datapb.DescribeIndexResponse
dropIndexResp *commonpb.Status
getIndexStateResp *datapb.GetIndexStateResponse
getIndexBuildProgressResp *datapb.GetIndexBuildProgressResponse
getSegmentIndexStateResp *datapb.GetSegmentIndexStateResponse
getIndexInfosResp *datapb.GetIndexInfoResponse
}
func (m *MockDataCoord) Init() error {
@ -209,14 +214,6 @@ func (m *MockDataCoord) UpdateChannelCheckpoint(ctx context.Context, req *datapb
return m.updateChanPos, m.err
}
func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
return m.acquireSegLockResp, m.err
}
func (m *MockDataCoord) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return m.releaseSegLockResp, m.err
}
func (m *MockDataCoord) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
return m.addSegmentResp, m.err
}
@ -239,6 +236,34 @@ func (m *MockDataCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHeal
}, nil
}
func (m *MockDataCoord) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
return m.createIndexResp, m.err
}
func (m *MockDataCoord) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return m.describeIndexResp, m.err
}
func (m *MockDataCoord) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
return m.getIndexInfosResp, m.err
}
func (m *MockDataCoord) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
return m.getIndexStateResp, m.err
}
func (m *MockDataCoord) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
return m.getIndexBuildProgressResp, m.err
}
func (m *MockDataCoord) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return m.getSegmentIndexStateResp, m.err
}
func (m *MockDataCoord) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return m.dropIndexResp, m.err
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func Test_NewServer(t *testing.T) {
paramtable.Init()
@ -248,9 +273,9 @@ func Test_NewServer(t *testing.T) {
t.Run("Run", func(t *testing.T) {
server.dataCoord = &MockDataCoord{}
indexCoord := mocks.NewMockIndexCoord(t)
indexCoord.EXPECT().Init().Return(nil)
server.indexCoord = indexCoord
//indexCoord := mocks.NewMockIndexCoord(t)
//indexCoord.EXPECT().Init().Return(nil)
//server.indexCoord = indexCoord
err := server.Run()
assert.Nil(t, err)
@ -496,28 +521,6 @@ func Test_NewServer(t *testing.T) {
assert.NotNil(t, resp)
})
t.Run("acquire segment reference lock", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
acquireSegLockResp: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}
resp, err := server.AcquireSegmentLock(ctx, nil)
assert.Nil(t, err)
assert.NotNil(t, resp)
})
t.Run("release segment reference lock", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
releaseSegLockResp: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}
resp, err := server.ReleaseSegmentLock(ctx, nil)
assert.Nil(t, err)
assert.NotNil(t, resp)
})
t.Run("save import segment", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
addSegmentResp: &commonpb.Status{
@ -567,6 +570,69 @@ func Test_NewServer(t *testing.T) {
assert.Equal(t, true, ret.IsHealthy)
})
t.Run("CreateIndex", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
createIndexResp: &commonpb.Status{},
}
ret, err := server.CreateIndex(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
t.Run("DescribeIndex", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
describeIndexResp: &datapb.DescribeIndexResponse{},
}
ret, err := server.DescribeIndex(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
t.Run("DropIndex", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
dropIndexResp: &commonpb.Status{},
}
ret, err := server.DropIndex(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
t.Run("GetIndexState", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
getIndexStateResp: &datapb.GetIndexStateResponse{},
}
ret, err := server.GetIndexState(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
t.Run("GetIndexBuildProgress", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
getIndexBuildProgressResp: &datapb.GetIndexBuildProgressResponse{},
}
ret, err := server.GetIndexBuildProgress(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
t.Run("GetSegmentIndexState", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
getSegmentIndexStateResp: &datapb.GetSegmentIndexStateResponse{},
}
ret, err := server.GetSegmentIndexState(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
t.Run("GetIndexInfos", func(t *testing.T) {
server.dataCoord = &MockDataCoord{
getIndexInfosResp: &datapb.GetIndexInfoResponse{},
}
ret, err := server.GetIndexInfos(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, ret)
})
err := server.Stop()
assert.Nil(t, err)
}

View File

@ -23,11 +23,10 @@ import (
"time"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/util/mock"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/util/mock"
"github.com/stretchr/testify/assert"
"google.golang.org/grpc"
)
func Test_NewClient(t *testing.T) {

View File

@ -40,7 +40,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
"github.com/milvus-io/milvus/internal/distributed/proxy/httpserver"
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
@ -87,7 +86,6 @@ type Server struct {
rootCoordClient types.RootCoord
dataCoordClient types.DataCoord
queryCoordClient types.QueryCoord
indexCoordClient types.IndexCoord
tracer opentracing.Tracer
closer io.Closer
@ -419,35 +417,6 @@ func (s *Server) init() error {
s.proxy.SetDataCoordClient(s.dataCoordClient)
log.Debug("set DataCoord client for Proxy done")
if s.indexCoordClient == nil {
var err error
log.Debug("create IndexCoord client for Proxy")
s.indexCoordClient, err = icc.NewClient(s.ctx, proxy.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
if err != nil {
log.Warn("failed to create IndexCoord client for Proxy", zap.Error(err))
return err
}
log.Debug("create IndexCoord client for Proxy done")
}
log.Debug("init IndexCoord client for Proxy")
if err := s.indexCoordClient.Init(); err != nil {
log.Warn("failed to init IndexCoord client for Proxy", zap.Error(err))
return err
}
log.Debug("init IndexCoord client for Proxy done")
log.Debug("Proxy wait for IndexCoord to be healthy")
if err := funcutil.WaitForComponentHealthy(s.ctx, s.indexCoordClient, "IndexCoord", 1000000, time.Millisecond*200); err != nil {
log.Warn("Proxy failed to wait for IndexCoord to be healthy", zap.Error(err))
return err
}
log.Debug("Proxy wait for IndexCoord to be healthy done")
log.Debug("set IndexCoord client for Proxy")
s.proxy.SetIndexCoordClient(s.indexCoordClient)
log.Debug("set IndexCoord client for Proxy done")
if s.queryCoordClient == nil {
var err error
log.Debug("create QueryCoord client for Proxy")

View File

@ -28,30 +28,28 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
milvusmock "github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/uniquegenerator"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
clientv3 "go.etcd.io/etcd/client/v3"
"google.golang.org/grpc/health/grpc_health_v1"
)
func TestMain(m *testing.M) {
@ -289,72 +287,7 @@ func (m *MockRootCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHeal
}, nil
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockIndexCoord struct {
MockBase
initErr error
startErr error
regErr error
stopErr error
}
func (m *MockIndexCoord) Init() error {
return m.initErr
}
func (m *MockIndexCoord) Start() error {
return m.startErr
}
func (m *MockIndexCoord) Stop() error {
return m.stopErr
}
func (m *MockIndexCoord) Register() error {
return m.regErr
}
func (m *MockIndexCoord) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockIndexCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockIndexCoord) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return nil, nil
}
func (m *MockIndexCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
return nil, nil
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockQueryCoord struct {
MockBase
initErr error
@ -627,7 +560,35 @@ func (m *MockDataCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHeal
return nil, nil
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func (m *MockDataCoord) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
return nil, nil
}
func (m *MockDataCoord) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
return nil, nil
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockProxy struct {
MockBase
err error
@ -1077,7 +1038,6 @@ func Test_NewServer(t *testing.T) {
server.proxy = &MockProxy{}
server.rootCoordClient = &MockRootCoord{}
server.indexCoordClient = &MockIndexCoord{}
server.queryCoordClient = &MockQueryCoord{}
server.dataCoordClient = &MockDataCoord{}
@ -1402,7 +1362,6 @@ func TestServer_Check(t *testing.T) {
mockProxy := &MockProxy{}
server.proxy = mockProxy
server.rootCoordClient = &MockRootCoord{}
server.indexCoordClient = &MockIndexCoord{}
server.queryCoordClient = &MockQueryCoord{}
server.dataCoordClient = &MockDataCoord{}
@ -1458,7 +1417,6 @@ func TestServer_Watch(t *testing.T) {
mockProxy := &MockProxy{}
server.proxy = mockProxy
server.rootCoordClient = &MockRootCoord{}
server.indexCoordClient = &MockIndexCoord{}
server.queryCoordClient = &MockQueryCoord{}
server.dataCoordClient = &MockDataCoord{}
@ -1523,7 +1481,6 @@ func Test_NewServer_HTTPServer_Enabled(t *testing.T) {
server.proxy = &MockProxy{}
server.rootCoordClient = &MockRootCoord{}
server.indexCoordClient = &MockIndexCoord{}
server.queryCoordClient = &MockQueryCoord{}
server.dataCoordClient = &MockDataCoord{}
@ -1552,7 +1509,6 @@ func getServer(t *testing.T) *Server {
server.proxy = &MockProxy{}
server.rootCoordClient = &MockRootCoord{}
server.indexCoordClient = &MockIndexCoord{}
server.queryCoordClient = &MockQueryCoord{}
server.dataCoordClient = &MockDataCoord{}
return server

View File

@ -34,7 +34,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/internalpb"
@ -64,9 +63,8 @@ type Server struct {
etcdCli *clientv3.Client
dataCoord types.DataCoord
rootCoord types.RootCoord
indexCoord types.IndexCoord
dataCoord types.DataCoord
rootCoord types.RootCoord
closer io.Closer
}
@ -195,37 +193,6 @@ func (s *Server) init() error {
}
log.Debug("QueryCoord report DataCoord ready")
// --- IndexCoord ---
if s.indexCoord == nil {
s.indexCoord, err = icc.NewClient(s.loopCtx, qc.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil {
log.Debug("QueryCoord try to new IndexCoord client failed", zap.Error(err))
panic(err)
}
}
if err := s.indexCoord.Init(); err != nil {
log.Debug("QueryCoord IndexCoordClient Init failed", zap.Error(err))
panic(err)
}
if err := s.indexCoord.Start(); err != nil {
log.Debug("QueryCoord IndexCoordClient Start failed", zap.Error(err))
panic(err)
}
// wait IndexCoord healthy
log.Debug("QueryCoord try to wait for IndexCoord ready")
err = funcutil.WaitForComponentHealthy(s.loopCtx, s.indexCoord, "IndexCoord", 1000000, time.Millisecond*200)
if err != nil {
log.Debug("QueryCoord wait for IndexCoord ready failed", zap.Error(err))
panic(err)
}
log.Debug("QueryCoord report IndexCoord is ready")
if err := s.SetIndexCoord(s.indexCoord); err != nil {
panic(err)
}
s.queryCoord.UpdateStateCode(commonpb.StateCode_Initializing)
log.Debug("QueryCoord", zap.Any("State", commonpb.StateCode_Initializing))
if err := s.queryCoord.Init(); err != nil {

View File

@ -230,40 +230,6 @@ func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*milvuspb.Compo
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockIndexCoord struct {
types.IndexCoord
initErr error
startErr error
stopErr error
regErr error
stateErr commonpb.ErrorCode
}
func (m *MockIndexCoord) Init() error {
return m.initErr
}
func (m *MockIndexCoord) Start() error {
return m.startErr
}
func (m *MockIndexCoord) Stop() error {
return m.stopErr
}
func (m *MockIndexCoord) Register() error {
return m.regErr
}
func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: &commonpb.Status{ErrorCode: m.stateErr},
}, nil
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func TestMain(m *testing.M) {
paramtable.Init()
code := m.Run()
@ -300,15 +266,10 @@ func Test_NewServer(t *testing.T) {
stateErr: commonpb.ErrorCode_Success,
}
mic := &MockIndexCoord{
stateErr: commonpb.ErrorCode_Success,
}
t.Run("Run", func(t *testing.T) {
server.queryCoord = mqc
server.dataCoord = mdc
server.rootCoord = mrc
server.indexCoord = mic
err = server.Run()
assert.Nil(t, err)
@ -443,7 +404,6 @@ func TestServer_Run2(t *testing.T) {
server.rootCoord = &MockRootCoord{
initErr: errors.New("error"),
}
server.indexCoord = &MockIndexCoord{}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.Nil(t, err)
@ -459,7 +419,6 @@ func TestServer_Run3(t *testing.T) {
server.rootCoord = &MockRootCoord{
startErr: errors.New("error"),
}
server.indexCoord = &MockIndexCoord{}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.Nil(t, err)
@ -477,7 +436,6 @@ func TestServer_Run4(t *testing.T) {
server.dataCoord = &MockDataCoord{
initErr: errors.New("error"),
}
server.indexCoord = &MockIndexCoord{}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.Nil(t, err)
@ -494,7 +452,6 @@ func TestServer_Run5(t *testing.T) {
server.dataCoord = &MockDataCoord{
startErr: errors.New("error"),
}
server.indexCoord = &MockIndexCoord{}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.Nil(t, err)

View File

@ -47,7 +47,6 @@ import (
"github.com/milvus-io/milvus/internal/util/trace"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
icc "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
qcc "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
)
@ -64,10 +63,8 @@ type Server struct {
etcdCli *clientv3.Client
dataCoord types.DataCoord
indexCoord types.IndexCoord
queryCoord types.QueryCoord
newIndexCoordClient func(string, *clientv3.Client) types.IndexCoord
newDataCoordClient func(string, *clientv3.Client) types.DataCoord
newQueryCoordClient func(string, *clientv3.Client) types.QueryCoord
@ -118,13 +115,7 @@ func (s *Server) setClient() {
}
return dsClient
}
s.newIndexCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.IndexCoord {
isClient, err := icc.NewClient(s.ctx, metaRootPath, etcdCli)
if err != nil {
panic(err)
}
return isClient
}
s.newQueryCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.QueryCoord {
qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdCli)
if err != nil {
@ -190,14 +181,7 @@ func (s *Server) init() error {
}
s.dataCoord = dataCoord
}
if s.newIndexCoordClient != nil {
log.Debug("RootCoord start to create IndexCoord client")
indexCoord := s.newIndexCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err := s.rootCoord.SetIndexCoord(indexCoord); err != nil {
panic(err)
}
s.indexCoord = indexCoord
}
if s.newQueryCoordClient != nil {
log.Debug("RootCoord start to create QueryCoord client")
queryCoord := s.newQueryCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
@ -286,11 +270,6 @@ func (s *Server) Stop() error {
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
if s.indexCoord != nil {
if err := s.indexCoord.Stop(); err != nil {
log.Error("Failed to close indexCoord client", zap.Error(err))
}
}
if s.dataCoord != nil {
if err := s.dataCoord.Stop(); err != nil {
log.Error("Failed to close dataCoord client", zap.Error(err))

View File

@ -168,9 +168,6 @@ func TestRun(t *testing.T) {
svr.newDataCoordClient = func(string, *clientv3.Client) types.DataCoord {
return &mockDataCoord{}
}
svr.newIndexCoordClient = func(string, *clientv3.Client) types.IndexCoord {
return &mockIndex{}
}
svr.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord {
return &mockQuery{}
}

View File

@ -61,14 +61,14 @@ func newGarbageCollector(ctx context.Context, meta *metaTable, chunkManager stor
}
func (gc *garbageCollector) Start() {
gc.wg.Add(1)
go gc.recycleUnusedIndexes()
gc.wg.Add(1)
go gc.recycleUnusedSegIndexes()
gc.wg.Add(1)
go gc.recycleUnusedIndexFiles()
//gc.wg.Add(1)
//go gc.recycleUnusedIndexes()
//
//gc.wg.Add(1)
//go gc.recycleUnusedSegIndexes()
//
//gc.wg.Add(1)
//go gc.recycleUnusedIndexFiles()
}
func (gc *garbageCollector) Stop() {

File diff suppressed because it is too large Load Diff

View File

@ -243,12 +243,12 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
}
// acquire lock
if err := ib.ic.tryAcquireSegmentReferLock(ib.ctx, buildID, nodeID, []UniqueID{meta.SegmentID}); err != nil {
log.Ctx(ib.ctx).Warn("index builder acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
return false
}
//if err := ib.ic.tryAcquireSegmentReferLock(ib.ctx, buildID, nodeID, []UniqueID{meta.SegmentID}); err != nil {
// log.Ctx(ib.ctx).Warn("index builder acquire segment reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Error(err))
// updateStateFunc(buildID, indexTaskRetry)
// return false
//}
info, err := ib.ic.pullSegmentInfo(ib.ctx, meta.SegmentID)
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID),
@ -462,12 +462,12 @@ func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
}
func (ib *indexBuilder) releaseLockAndResetNode(buildID UniqueID, nodeID UniqueID) error {
if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Ctx(ib.ctx).Warn("index builder try to release reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
return err
}
//if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// // release lock failed, no need to modify state, wait to retry
// log.Ctx(ib.ctx).Warn("index builder try to release reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Error(err))
// return err
//}
if err := ib.meta.ResetNodeID(buildID); err != nil {
log.Ctx(ib.ctx).Warn("index builder try to reset nodeID failed", zap.Error(err))
return err
@ -478,13 +478,13 @@ func (ib *indexBuilder) releaseLockAndResetNode(buildID UniqueID, nodeID UniqueI
}
func (ib *indexBuilder) releaseLockAndResetTask(buildID UniqueID, nodeID UniqueID) error {
if nodeID != 0 {
if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
log.Ctx(ib.ctx).Warn("index builder try to release reference lock failed", zap.Error(err))
return err
}
}
//if nodeID != 0 {
//if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// // release lock failed, no need to modify state, wait to retry
// log.Ctx(ib.ctx).Warn("index builder try to release reference lock failed", zap.Error(err))
// return err
//}
//}
if err := ib.meta.ResetMeta(buildID); err != nil {
log.Ctx(ib.ctx).Warn("index builder try to reset task failed", zap.Error(err))
return err

View File

@ -19,7 +19,6 @@ package indexcoord
import (
"context"
"errors"
"sync"
"testing"
"time"
@ -585,18 +584,18 @@ func TestIndexBuilder_Error(t *testing.T) {
ib.process(buildID)
})
t.Run("acquire lock fail", func(t *testing.T) {
ib.tasks[buildID] = indexTaskInit
ib.meta = createMetaTable(&indexcoord.Catalog{
Txn: NewMockEtcdKV(),
})
dataMock := NewDataCoordMock()
dataMock.CallAcquireSegmentLock = func(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
return nil, errors.New("error")
}
ib.ic.dataCoordClient = dataMock
ib.process(buildID)
})
//t.Run("acquire lock fail", func(t *testing.T) {
// ib.tasks[buildID] = indexTaskInit
// ib.meta = createMetaTable(&indexcoord.Catalog{
// Txn: NewMockEtcdKV(),
// })
// dataMock := NewDataCoordMock()
// dataMock.CallAcquireSegmentLock = func(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
// return nil, errors.New("error")
// }
// ib.ic.dataCoordClient = dataMock
// ib.process(buildID)
//})
t.Run("get segment info error", func(t *testing.T) {
ib.tasks[buildID] = indexTaskInit
@ -1124,62 +1123,62 @@ func Test_indexBuilder_getTaskState(t *testing.T) {
})
}
func Test_indexBuilder_releaseLockAndResetNode_error(t *testing.T) {
Params.Init()
ctx, cancel := context.WithCancel(context.Background())
ib := &indexBuilder{
ctx: ctx,
cancel: cancel,
tasks: map[int64]indexTaskState{
buildID: indexTaskInit,
},
meta: createMetaTable(&indexcoord.Catalog{Txn: NewMockEtcdKV()}),
ic: &IndexCoord{
dataCoordClient: &DataCoordMock{
CallReleaseSegmentLock: func(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return nil, errors.New("error")
},
},
},
}
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
err := ib.releaseLockAndResetNode(buildID, nodeID)
assert.Error(t, err)
}()
time.Sleep(time.Second)
ib.cancel()
wg.Wait()
}
//func Test_indexBuilder_releaseLockAndResetNode_error(t *testing.T) {
// Params.Init()
// ctx, cancel := context.WithCancel(context.Background())
// ib := &indexBuilder{
// ctx: ctx,
// cancel: cancel,
// tasks: map[int64]indexTaskState{
// buildID: indexTaskInit,
// },
// meta: createMetaTable(&indexcoord.Catalog{Txn: NewMockEtcdKV()}),
// ic: &IndexCoord{
// dataCoordClient: &DataCoordMock{
// CallReleaseSegmentLock: func(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
// return nil, errors.New("error")
// },
// },
// },
// }
// var wg sync.WaitGroup
// wg.Add(1)
// go func() {
// defer wg.Done()
// err := ib.releaseLockAndResetNode(buildID, nodeID)
// assert.Error(t, err)
// }()
// time.Sleep(time.Second)
// ib.cancel()
// wg.Wait()
//}
func Test_indexBuilder_releaseLockAndResetTask_error(t *testing.T) {
Params.Init()
ctx, cancel := context.WithCancel(context.Background())
ib := &indexBuilder{
ctx: ctx,
cancel: cancel,
tasks: map[int64]indexTaskState{
buildID: indexTaskInit,
},
meta: createMetaTable(&indexcoord.Catalog{Txn: NewMockEtcdKV()}),
ic: &IndexCoord{
dataCoordClient: &DataCoordMock{
CallReleaseSegmentLock: func(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return nil, errors.New("error")
},
},
},
}
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
err := ib.releaseLockAndResetTask(buildID, nodeID)
assert.Error(t, err)
}()
time.Sleep(time.Second)
ib.cancel()
wg.Wait()
}
//func Test_indexBuilder_releaseLockAndResetTask_error(t *testing.T) {
// Params.Init()
// ctx, cancel := context.WithCancel(context.Background())
// ib := &indexBuilder{
// ctx: ctx,
// cancel: cancel,
// tasks: map[int64]indexTaskState{
// buildID: indexTaskInit,
// },
// meta: createMetaTable(&indexcoord.Catalog{Txn: NewMockEtcdKV()}),
// ic: &IndexCoord{
// dataCoordClient: &DataCoordMock{
// CallReleaseSegmentLock: func(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
// return nil, errors.New("error")
// },
// },
// },
// }
// var wg sync.WaitGroup
// wg.Add(1)
// go func() {
// defer wg.Done()
// err := ib.releaseLockAndResetTask(buildID, nodeID)
// assert.Error(t, err)
// }()
// time.Sleep(time.Second)
// ib.cancel()
// wg.Wait()
//}

View File

@ -28,6 +28,9 @@ import (
"syscall"
"time"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/util/metautil"
@ -45,7 +48,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/kv"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/metrics"
@ -58,7 +60,6 @@ import (
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
@ -298,7 +299,7 @@ func (i *IndexCoord) Start() error {
if i.enableActiveStandBy {
i.activateFunc = func() {
log.Info("IndexCoord switch from standby to active, reload the KV")
i.metaTable.reloadFromKV()
//i.metaTable.reloadFromKV()
i.UpdateStateCode(commonpb.StateCode_Healthy)
}
i.UpdateStateCode(commonpb.StateCode_StandBy)
@ -1129,23 +1130,23 @@ func (i *IndexCoord) tryAcquireSegmentReferLock(ctx context.Context, buildID Uni
// IndexCoord use buildID instead of taskID.
log.Info("try to acquire segment reference lock", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs))
ctx1, cancel := context.WithTimeout(ctx, reqTimeoutInterval)
defer cancel()
status, err := i.dataCoordClient.AcquireSegmentLock(ctx1, &datapb.AcquireSegmentLockRequest{
TaskID: buildID,
NodeID: nodeID,
SegmentIDs: segIDs,
})
if err != nil {
log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(err))
return err
}
if status.ErrorCode != commonpb.ErrorCode_Success {
log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(errors.New(status.Reason)))
return errors.New(status.Reason)
}
//ctx1, cancel := context.WithTimeout(ctx, reqTimeoutInterval)
//defer cancel()
//status, err := i.dataCoordClient.AcquireSegmentLock(ctx1, &datapb.AcquireSegmentLockRequest{
// TaskID: buildID,
// NodeID: nodeID,
// SegmentIDs: segIDs,
//})
//if err != nil {
// log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(err))
// return err
//}
//if status.ErrorCode != commonpb.ErrorCode_Success {
// log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Int64s("segIDs", segIDs), zap.Error(errors.New(status.Reason)))
// return errors.New(status.Reason)
//}
log.Info("try to acquire segment reference lock success", zap.Int64("buildID", buildID),
zap.Int64("ndoeID", nodeID), zap.Int64s("segIDs", segIDs))
return nil
@ -1154,27 +1155,27 @@ func (i *IndexCoord) tryAcquireSegmentReferLock(ctx context.Context, buildID Uni
func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, buildID UniqueID, nodeID UniqueID) error {
log.Info("IndexCoord tryReleaseSegmentReferLock", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
releaseLock := func() error {
ctx1, cancel := context.WithTimeout(ctx, reqTimeoutInterval)
defer cancel()
status, err := i.dataCoordClient.ReleaseSegmentLock(ctx1, &datapb.ReleaseSegmentLockRequest{
TaskID: buildID,
NodeID: nodeID,
})
if err != nil {
return err
}
if status.ErrorCode != commonpb.ErrorCode_Success {
return errors.New(status.Reason)
}
return nil
}
err := retry.Do(ctx, releaseLock, retry.Attempts(100))
if err != nil {
log.Error("IndexCoord try to release segment reference lock failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
return err
}
//releaseLock := func() error {
// ctx1, cancel := context.WithTimeout(ctx, reqTimeoutInterval)
// defer cancel()
// status, err := i.dataCoordClient.ReleaseSegmentLock(ctx1, &datapb.ReleaseSegmentLockRequest{
// TaskID: buildID,
// NodeID: nodeID,
// })
// if err != nil {
// return err
// }
// if status.ErrorCode != commonpb.ErrorCode_Success {
// return errors.New(status.Reason)
// }
// return nil
//}
//err := retry.Do(ctx, releaseLock, retry.Attempts(100))
//if err != nil {
// log.Error("IndexCoord try to release segment reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Error(err))
// return err
//}
log.Info("IndexCoord tryReleaseSegmentReferLock successfully", zap.Int64("buildID", buildID),
zap.Int64("ndoeID", nodeID))
return nil

View File

@ -1,3 +1,19 @@
// 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 indexnode
import (

View File

@ -117,6 +117,18 @@ type DataCoordCatalog interface {
ListChannelCheckpoint(ctx context.Context) (map[string]*internalpb.MsgPosition, error)
SaveChannelCheckpoint(ctx context.Context, vChannel string, pos *internalpb.MsgPosition) error
DropChannelCheckpoint(ctx context.Context, vChannel string) error
CreateIndex(ctx context.Context, index *model.Index) error
ListIndexes(ctx context.Context) ([]*model.Index, error)
AlterIndex(ctx context.Context, newIndex *model.Index) error
AlterIndexes(ctx context.Context, newIndexes []*model.Index) error
DropIndex(ctx context.Context, collID, dropIdxID typeutil.UniqueID) error
CreateSegmentIndex(ctx context.Context, segIdx *model.SegmentIndex) error
ListSegmentIndexes(ctx context.Context) ([]*model.SegmentIndex, error)
AlterSegmentIndex(ctx context.Context, newSegIndex *model.SegmentIndex) error
AlterSegmentIndexes(ctx context.Context, newSegIdxes []*model.SegmentIndex) error
DropSegmentIndex(ctx context.Context, collID, partID, segID, buildID typeutil.UniqueID) error
}
type IndexCoordCatalog interface {

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"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/internalpb"
"github.com/milvus-io/milvus/internal/storage"
@ -171,15 +172,6 @@ func (kc *Catalog) AlterSegment(ctx context.Context, newSegment *datapb.SegmentI
return err
}
maps.Copy(kvs, segmentKvs)
if newSegment.State == commonpb.SegmentState_Flushed && oldSegment.State != commonpb.SegmentState_Flushed {
flushSegKey := buildFlushedSegmentPath(newSegment.GetCollectionID(), newSegment.GetPartitionID(), newSegment.GetID())
newSeg := &datapb.SegmentInfo{ID: newSegment.GetID()}
segBytes, err := marshalSegmentInfo(newSeg)
if err != nil {
return err
}
kvs[flushSegKey] = segBytes
}
return kc.Txn.MultiSave(kvs)
}
@ -247,16 +239,6 @@ func (kc *Catalog) AlterSegmentsAndAddNewSegment(ctx context.Context, segments [
return err
}
maps.Copy(kvs, segmentKvs)
} else {
// should be a faked segment, we create flush path directly here
flushSegKey := buildFlushedSegmentPath(newSegment.GetCollectionID(), newSegment.GetPartitionID(), newSegment.GetID())
clonedSegment := proto.Clone(newSegment).(*datapb.SegmentInfo)
clonedSegment.IsFake = true
segBytes, err := marshalSegmentInfo(clonedSegment)
if err != nil {
return err
}
kvs[flushSegKey] = segBytes
}
}
return kc.Txn.MultiSave(kvs)
@ -435,6 +417,142 @@ func (kc *Catalog) unmarshalBinlog(binlogType storage.BinlogType, collectionID,
return result, nil
}
func (kc *Catalog) CreateIndex(ctx context.Context, index *model.Index) error {
key := BuildIndexKey(index.CollectionID, index.IndexID)
value, err := proto.Marshal(model.MarshalIndexModel(index))
if err != nil {
return err
}
err = kc.Txn.Save(key, string(value))
if err != nil {
return err
}
return nil
}
func (kc *Catalog) ListIndexes(ctx context.Context) ([]*model.Index, error) {
_, values, err := kc.Txn.LoadWithPrefix(util.FieldIndexPrefix)
if err != nil {
log.Error("list index meta fail", zap.String("prefix", util.FieldIndexPrefix), zap.Error(err))
return nil, err
}
indexes := make([]*model.Index, 0)
for _, value := range values {
meta := &datapb.FieldIndex{}
err = proto.Unmarshal([]byte(value), meta)
if err != nil {
log.Warn("unmarshal index info failed", zap.Error(err))
return nil, err
}
indexes = append(indexes, model.UnmarshalIndexModel(meta))
}
return indexes, nil
}
func (kc *Catalog) AlterIndex(ctx context.Context, index *model.Index) error {
return kc.CreateIndex(ctx, index)
}
func (kc *Catalog) AlterIndexes(ctx context.Context, indexes []*model.Index) error {
kvs := make(map[string]string)
for _, index := range indexes {
key := BuildIndexKey(index.CollectionID, index.IndexID)
value, err := proto.Marshal(model.MarshalIndexModel(index))
if err != nil {
return err
}
kvs[key] = string(value)
}
return kc.Txn.MultiSave(kvs)
}
func (kc *Catalog) DropIndex(ctx context.Context, collID typeutil.UniqueID, dropIdxID typeutil.UniqueID) error {
key := BuildIndexKey(collID, dropIdxID)
err := kc.Txn.Remove(key)
if err != nil {
log.Error("drop collection index meta fail", zap.Int64("collectionID", collID),
zap.Int64("indexID", dropIdxID), zap.Error(err))
return err
}
return nil
}
func (kc *Catalog) CreateSegmentIndex(ctx context.Context, segIdx *model.SegmentIndex) error {
key := BuildSegmentIndexKey(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.BuildID)
value, err := proto.Marshal(model.MarshalSegmentIndexModel(segIdx))
if err != nil {
return err
}
err = kc.Txn.Save(key, string(value))
if err != nil {
log.Error("failed to save segment index meta in etcd", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("segmentID", segIdx.SegmentID), zap.Error(err))
return err
}
return nil
}
func (kc *Catalog) ListSegmentIndexes(ctx context.Context) ([]*model.SegmentIndex, error) {
_, values, err := kc.Txn.LoadWithPrefix(util.SegmentIndexPrefix)
if err != nil {
log.Error("list segment index meta fail", zap.String("prefix", util.SegmentIndexPrefix), zap.Error(err))
return nil, err
}
segIndexes := make([]*model.SegmentIndex, 0)
for _, value := range values {
segmentIndexInfo := &datapb.SegmentIndex{}
err = proto.Unmarshal([]byte(value), segmentIndexInfo)
if err != nil {
log.Warn("unmarshal segment index info failed", zap.Error(err))
return segIndexes, err
}
segIndexes = append(segIndexes, model.UnmarshalSegmentIndexModel(segmentIndexInfo))
}
return segIndexes, nil
}
func (kc *Catalog) AlterSegmentIndex(ctx context.Context, segIdx *model.SegmentIndex) error {
return kc.CreateSegmentIndex(ctx, segIdx)
}
func (kc *Catalog) AlterSegmentIndexes(ctx context.Context, segIdxes []*model.SegmentIndex) error {
kvs := make(map[string]string)
for _, segIdx := range segIdxes {
key := BuildSegmentIndexKey(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.BuildID)
value, err := proto.Marshal(model.MarshalSegmentIndexModel(segIdx))
if err != nil {
return err
}
kvs[key] = string(value)
}
return kc.Txn.MultiSave(kvs)
}
func (kc *Catalog) DropSegmentIndex(ctx context.Context, collID, partID, segID, buildID typeutil.UniqueID) error {
key := BuildSegmentIndexKey(collID, partID, segID, buildID)
err := kc.Txn.Remove(key)
if err != nil {
log.Error("drop segment index meta fail", zap.Int64("buildID", buildID), zap.Error(err))
return err
}
return nil
}
func fillLogPathByLogID(chunkManagerRootPath string, binlogType storage.BinlogType, collectionID, partitionID,
segmentID typeutil.UniqueID, fieldBinlog *datapb.FieldBinlog) error {
for _, binlog := range fieldBinlog.Binlogs {
@ -672,7 +790,8 @@ func buildFieldStatslogPath(collectionID typeutil.UniqueID, partitionID typeutil
return fmt.Sprintf("%s/%d/%d/%d/%d", SegmentStatslogPathPrefix, collectionID, partitionID, segmentID, fieldID)
}
// buildFlushedSegmentPath common logic mapping segment info to corresponding key of IndexCoord in kv store
//buildFlushedSegmentPath common logic mapping segment info to corresponding key of IndexCoord in kv store
// TODO @cai.zhang: remove this
func buildFlushedSegmentPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string {
return fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, collectionID, partitionID, segmentID)
}
@ -697,3 +816,11 @@ func buildChannelRemovePath(channel string) string {
func buildChannelCPKey(vChannel string) string {
return fmt.Sprintf("%s/%s", ChannelCheckpointPrefix, vChannel)
}
func BuildIndexKey(collectionID, indexID int64) string {
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", util.SegmentIndexPrefix, collectionID, partitionID, segmentID, buildID)
}

View File

@ -1,3 +1,19 @@
// 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 datacoord
import (
@ -14,6 +30,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/metautil"
@ -26,6 +43,7 @@ type MockedTxnKV struct {
loadWithPrefix func(key string) ([]string, []string, error)
load func(key string) (string, error)
multiRemove func(keys []string) error
remove func(key string) error
}
var (
@ -176,6 +194,10 @@ func (mc *MockedTxnKV) Load(key string) (string, error) {
return mc.load(key)
}
func (mc *MockedTxnKV) Remove(key string) error {
return mc.remove(key)
}
func Test_ListSegments(t *testing.T) {
t.Run("load failed", func(t *testing.T) {
txn := &MockedTxnKV{}
@ -774,3 +796,379 @@ func verifySavedKvsForDroppedSegment(t *testing.T, savedKvs map[string]string) {
assert.True(t, ok)
verifySegmentInfo2(t, []byte(ret))
}
func TestCatalog_CreateIndex(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateIndex(context.Background(), &model.Index{})
assert.NoError(t, err)
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateIndex(context.Background(), &model.Index{})
assert.Error(t, err)
})
}
func TestCatalog_ListIndexes(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
i := &datapb.FieldIndex{
IndexInfo: &datapb.IndexInfo{
CollectionID: 0,
FieldID: 0,
IndexName: "",
IndexID: 0,
TypeParams: nil,
IndexParams: nil,
},
Deleted: false,
CreateTime: 0,
}
v, err := proto.Marshal(i)
assert.NoError(t, err)
return []string{"1"}, []string{string(v)}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
indexes, err := catalog.ListIndexes(context.Background())
assert.NoError(t, err)
assert.Equal(t, 1, len(indexes))
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{}, []string{}, errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListIndexes(context.Background())
assert.Error(t, err)
})
t.Run("unmarshal failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{"1"}, []string{"invalid"}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListIndexes(context.Background())
assert.Error(t, err)
})
}
func TestCatalog_AlterIndex(t *testing.T) {
i := &model.Index{
CollectionID: 0,
FieldID: 0,
IndexID: 0,
IndexName: "",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
}
t.Run("add", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterIndex(context.Background(), i)
assert.NoError(t, err)
})
}
func TestCatalog_AlterIndexes(t *testing.T) {
i := &model.Index{
CollectionID: 0,
FieldID: 0,
IndexID: 0,
IndexName: "",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
}
txn := &MockedTxnKV{
multiSave: func(kvs map[string]string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterIndexes(context.Background(), []*model.Index{i})
assert.NoError(t, err)
}
func TestCatalog_DropIndex(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropIndex(context.Background(), 0, 0)
assert.NoError(t, err)
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropIndex(context.Background(), 0, 0)
assert.Error(t, err)
})
}
func TestCatalog_CreateSegmentIndex(t *testing.T) {
segIdx := &model.SegmentIndex{
SegmentID: 1,
CollectionID: 2,
PartitionID: 3,
NumRows: 1024,
IndexID: 4,
BuildID: 5,
NodeID: 6,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IndexVersion: 0,
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
}
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateSegmentIndex(context.Background(), segIdx)
assert.NoError(t, err)
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateSegmentIndex(context.Background(), segIdx)
assert.Error(t, err)
})
}
func TestCatalog_ListSegmentIndexes(t *testing.T) {
t.Run("success", func(t *testing.T) {
segIdx := &datapb.SegmentIndex{
CollectionID: 0,
PartitionID: 0,
SegmentID: 0,
NumRows: 0,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexVersion: 0,
State: 0,
FailReason: "",
IndexFileKeys: nil,
Deleted: false,
CreateTime: 0,
SerializeSize: 0,
}
v, err := proto.Marshal(segIdx)
assert.NoError(t, err)
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{"key"}, []string{string(v)}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
segIdxes, err := catalog.ListSegmentIndexes(context.Background())
assert.NoError(t, err)
assert.Equal(t, 1, len(segIdxes))
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{}, []string{}, errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListSegmentIndexes(context.Background())
assert.Error(t, err)
})
t.Run("unmarshal failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{"key"}, []string{"invalid"}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListSegmentIndexes(context.Background())
assert.Error(t, err)
})
}
func TestCatalog_AlterSegmentIndex(t *testing.T) {
segIdx := &model.SegmentIndex{
SegmentID: 0,
CollectionID: 0,
PartitionID: 0,
NumRows: 0,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexState: 0,
FailReason: "",
IndexVersion: 0,
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
}
t.Run("add", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterSegmentIndex(context.Background(), segIdx)
assert.NoError(t, err)
})
}
func TestCatalog_AlterSegmentIndexes(t *testing.T) {
segIdx := &model.SegmentIndex{
SegmentID: 0,
CollectionID: 0,
PartitionID: 0,
NumRows: 0,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexState: 0,
FailReason: "",
IndexVersion: 0,
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
}
t.Run("add", func(t *testing.T) {
txn := &MockedTxnKV{
multiSave: func(kvs map[string]string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterSegmentIndexes(context.Background(), []*model.SegmentIndex{segIdx})
assert.NoError(t, err)
})
}
func TestCatalog_DropSegmentIndex(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropSegmentIndex(context.Background(), 0, 0, 0, 0)
assert.NoError(t, err)
})
t.Run("fail", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropSegmentIndex(context.Background(), 0, 0, 0, 0)
assert.Error(t, err)
})
}

View File

@ -60,7 +60,19 @@ func (kc *Catalog) ListIndexes(ctx context.Context) ([]*model.Index, error) {
return nil, err
}
indexes = append(indexes, model.UnmarshalIndexModel(meta))
indexes = append(indexes, &model.Index{
TenantID: "",
CollectionID: meta.IndexInfo.CollectionID,
FieldID: meta.IndexInfo.FieldID,
IndexID: meta.IndexInfo.IndexID,
IndexName: meta.IndexInfo.IndexName,
IsDeleted: meta.Deleted,
CreateTime: meta.GetCreateTime(),
TypeParams: meta.IndexInfo.TypeParams,
IndexParams: meta.IndexInfo.IndexParams,
IsAutoIndex: meta.IndexInfo.IsAutoIndex,
UserIndexParams: meta.IndexInfo.UserIndexParams,
})
}
return indexes, nil
@ -127,10 +139,26 @@ func (kc *Catalog) ListSegmentIndexes(ctx context.Context) ([]*model.SegmentInde
err = proto.Unmarshal([]byte(value), segmentIndexInfo)
if err != nil {
log.Warn("unmarshal segment index info failed", zap.Error(err))
continue
return segIndexes, err
}
segIndexes = append(segIndexes, model.UnmarshalSegmentIndexModel(segmentIndexInfo))
segIndexes = append(segIndexes, &model.SegmentIndex{
SegmentID: segmentIndexInfo.SegmentID,
CollectionID: segmentIndexInfo.CollectionID,
PartitionID: segmentIndexInfo.PartitionID,
NumRows: segmentIndexInfo.NumRows,
IndexID: segmentIndexInfo.IndexID,
BuildID: segmentIndexInfo.BuildID,
NodeID: segmentIndexInfo.NodeID,
IndexVersion: segmentIndexInfo.IndexVersion,
IndexState: segmentIndexInfo.State,
FailReason: segmentIndexInfo.FailReason,
IsDeleted: segmentIndexInfo.Deleted,
CreateTime: segmentIndexInfo.CreateTime,
IndexFileKeys: segmentIndexInfo.IndexFileKeys,
IndexSize: segmentIndexInfo.SerializeSize,
WriteHandoff: segmentIndexInfo.WriteHandoff,
})
}
return segIndexes, nil

View File

@ -284,7 +284,7 @@ func TestCatalog_ListSegmentIndexes(t *testing.T) {
Txn: txn,
}
segIdxes, err := catalog.ListIndexes(context.Background())
segIdxes, err := catalog.ListSegmentIndexes(context.Background())
assert.NoError(t, err)
assert.Equal(t, 1, len(segIdxes))
})
@ -299,7 +299,7 @@ func TestCatalog_ListSegmentIndexes(t *testing.T) {
Txn: txn,
}
_, err := catalog.ListIndexes(context.Background())
_, err := catalog.ListSegmentIndexes(context.Background())
assert.Error(t, err)
})
@ -313,7 +313,7 @@ func TestCatalog_ListSegmentIndexes(t *testing.T) {
Txn: txn,
}
_, err := catalog.ListIndexes(context.Background())
_, err := catalog.ListSegmentIndexes(context.Background())
assert.Error(t, err)
})
}

View File

@ -3,7 +3,7 @@ package model
import (
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
type Index struct {
@ -20,7 +20,7 @@ type Index struct {
UserIndexParams []*commonpb.KeyValuePair
}
func UnmarshalIndexModel(indexInfo *indexpb.FieldIndex) *Index {
func UnmarshalIndexModel(indexInfo *datapb.FieldIndex) *Index {
if indexInfo == nil {
return nil
}
@ -39,13 +39,13 @@ func UnmarshalIndexModel(indexInfo *indexpb.FieldIndex) *Index {
}
}
func MarshalIndexModel(index *Index) *indexpb.FieldIndex {
func MarshalIndexModel(index *Index) *datapb.FieldIndex {
if index == nil {
return nil
}
return &indexpb.FieldIndex{
IndexInfo: &indexpb.IndexInfo{
return &datapb.FieldIndex{
IndexInfo: &datapb.IndexInfo{
CollectionID: index.CollectionID,
FieldID: index.FieldID,
IndexName: index.IndexName,

View File

@ -6,7 +6,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
var (
@ -27,8 +27,8 @@ var (
CreateTime: 1,
}
indexPb = &indexpb.FieldIndex{
IndexInfo: &indexpb.IndexInfo{
indexPb = &datapb.FieldIndex{
IndexInfo: &datapb.IndexInfo{
CollectionID: colID,
FieldID: fieldID,
IndexName: indexName,

View File

@ -3,7 +3,7 @@ package model
import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
type SegmentIndex struct {
@ -21,10 +21,11 @@ type SegmentIndex struct {
CreateTime uint64
IndexFileKeys []string
IndexSize uint64
WriteHandoff bool
// deprecated
WriteHandoff bool
}
func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex {
func UnmarshalSegmentIndexModel(segIndex *datapb.SegmentIndex) *SegmentIndex {
if segIndex == nil {
return nil
}
@ -48,12 +49,12 @@ func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex {
}
}
func MarshalSegmentIndexModel(segIdx *SegmentIndex) *indexpb.SegmentIndex {
func MarshalSegmentIndexModel(segIdx *SegmentIndex) *datapb.SegmentIndex {
if segIdx == nil {
return nil
}
return &indexpb.SegmentIndex{
return &datapb.SegmentIndex{
CollectionID: segIdx.CollectionID,
PartitionID: segIdx.PartitionID,
SegmentID: segIdx.SegmentID,

View File

@ -6,14 +6,14 @@ import (
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
var (
segmentID = int64(1)
buildID = int64(1)
segmentIdxPb = &indexpb.SegmentIndex{
segmentIdxPb = &datapb.SegmentIndex{
CollectionID: colID,
PartitionID: partID,
SegmentID: segmentID,

View File

@ -7,7 +7,7 @@ import (
commonpb "github.com/milvus-io/milvus-proto/go-api/commonpb"
indexpb "github.com/milvus-io/milvus/internal/proto/indexpb"
datapb "github.com/milvus-io/milvus/internal/proto/datapb"
internalpb "github.com/milvus-io/milvus/internal/proto/internalpb"
@ -77,11 +77,11 @@ func (_c *MockIndexCoord_CheckHealth_Call) Return(_a0 *milvuspb.CheckHealthRespo
}
// CreateIndex provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
func (_m *MockIndexCoord) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
var r0 *commonpb.Status
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.CreateIndexRequest) *commonpb.Status); ok {
if rf, ok := ret.Get(0).(func(context.Context, *datapb.CreateIndexRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
@ -90,7 +90,7 @@ func (_m *MockIndexCoord) CreateIndex(ctx context.Context, req *indexpb.CreateIn
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.CreateIndexRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.CreateIndexRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -106,14 +106,14 @@ type MockIndexCoord_CreateIndex_Call struct {
// CreateIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.CreateIndexRequest
// - req *datapb.CreateIndexRequest
func (_e *MockIndexCoord_Expecter) CreateIndex(ctx interface{}, req interface{}) *MockIndexCoord_CreateIndex_Call {
return &MockIndexCoord_CreateIndex_Call{Call: _e.mock.On("CreateIndex", ctx, req)}
}
func (_c *MockIndexCoord_CreateIndex_Call) Run(run func(ctx context.Context, req *indexpb.CreateIndexRequest)) *MockIndexCoord_CreateIndex_Call {
func (_c *MockIndexCoord_CreateIndex_Call) Run(run func(ctx context.Context, req *datapb.CreateIndexRequest)) *MockIndexCoord_CreateIndex_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.CreateIndexRequest))
run(args[0].(context.Context), args[1].(*datapb.CreateIndexRequest))
})
return _c
}
@ -124,20 +124,20 @@ func (_c *MockIndexCoord_CreateIndex_Call) Return(_a0 *commonpb.Status, _a1 erro
}
// DescribeIndex provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
func (_m *MockIndexCoord) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
ret := _m.Called(ctx, req)
var r0 *indexpb.DescribeIndexResponse
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DescribeIndexRequest) *indexpb.DescribeIndexResponse); ok {
var r0 *datapb.DescribeIndexResponse
if rf, ok := ret.Get(0).(func(context.Context, *datapb.DescribeIndexRequest) *datapb.DescribeIndexResponse); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.DescribeIndexResponse)
r0 = ret.Get(0).(*datapb.DescribeIndexResponse)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.DescribeIndexRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.DescribeIndexRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -153,29 +153,29 @@ type MockIndexCoord_DescribeIndex_Call struct {
// DescribeIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.DescribeIndexRequest
// - req *datapb.DescribeIndexRequest
func (_e *MockIndexCoord_Expecter) DescribeIndex(ctx interface{}, req interface{}) *MockIndexCoord_DescribeIndex_Call {
return &MockIndexCoord_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex", ctx, req)}
}
func (_c *MockIndexCoord_DescribeIndex_Call) Run(run func(ctx context.Context, req *indexpb.DescribeIndexRequest)) *MockIndexCoord_DescribeIndex_Call {
func (_c *MockIndexCoord_DescribeIndex_Call) Run(run func(ctx context.Context, req *datapb.DescribeIndexRequest)) *MockIndexCoord_DescribeIndex_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.DescribeIndexRequest))
run(args[0].(context.Context), args[1].(*datapb.DescribeIndexRequest))
})
return _c
}
func (_c *MockIndexCoord_DescribeIndex_Call) Return(_a0 *indexpb.DescribeIndexResponse, _a1 error) *MockIndexCoord_DescribeIndex_Call {
func (_c *MockIndexCoord_DescribeIndex_Call) Return(_a0 *datapb.DescribeIndexResponse, _a1 error) *MockIndexCoord_DescribeIndex_Call {
_c.Call.Return(_a0, _a1)
return _c
}
// DropIndex provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
func (_m *MockIndexCoord) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
var r0 *commonpb.Status
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DropIndexRequest) *commonpb.Status); ok {
if rf, ok := ret.Get(0).(func(context.Context, *datapb.DropIndexRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
@ -184,7 +184,7 @@ func (_m *MockIndexCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexR
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.DropIndexRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.DropIndexRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -200,14 +200,14 @@ type MockIndexCoord_DropIndex_Call struct {
// DropIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.DropIndexRequest
// - req *datapb.DropIndexRequest
func (_e *MockIndexCoord_Expecter) DropIndex(ctx interface{}, req interface{}) *MockIndexCoord_DropIndex_Call {
return &MockIndexCoord_DropIndex_Call{Call: _e.mock.On("DropIndex", ctx, req)}
}
func (_c *MockIndexCoord_DropIndex_Call) Run(run func(ctx context.Context, req *indexpb.DropIndexRequest)) *MockIndexCoord_DropIndex_Call {
func (_c *MockIndexCoord_DropIndex_Call) Run(run func(ctx context.Context, req *datapb.DropIndexRequest)) *MockIndexCoord_DropIndex_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.DropIndexRequest))
run(args[0].(context.Context), args[1].(*datapb.DropIndexRequest))
})
return _c
}
@ -264,20 +264,20 @@ func (_c *MockIndexCoord_GetComponentStates_Call) Return(_a0 *milvuspb.Component
}
// GetIndexBuildProgress provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
func (_m *MockIndexCoord) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
ret := _m.Called(ctx, req)
var r0 *indexpb.GetIndexBuildProgressResponse
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.GetIndexBuildProgressRequest) *indexpb.GetIndexBuildProgressResponse); ok {
var r0 *datapb.GetIndexBuildProgressResponse
if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetIndexBuildProgressRequest) *datapb.GetIndexBuildProgressResponse); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.GetIndexBuildProgressResponse)
r0 = ret.Get(0).(*datapb.GetIndexBuildProgressResponse)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.GetIndexBuildProgressRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetIndexBuildProgressRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -293,38 +293,38 @@ type MockIndexCoord_GetIndexBuildProgress_Call struct {
// GetIndexBuildProgress is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexBuildProgressRequest
// - req *datapb.GetIndexBuildProgressRequest
func (_e *MockIndexCoord_Expecter) GetIndexBuildProgress(ctx interface{}, req interface{}) *MockIndexCoord_GetIndexBuildProgress_Call {
return &MockIndexCoord_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress", ctx, req)}
}
func (_c *MockIndexCoord_GetIndexBuildProgress_Call) Run(run func(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest)) *MockIndexCoord_GetIndexBuildProgress_Call {
func (_c *MockIndexCoord_GetIndexBuildProgress_Call) Run(run func(ctx context.Context, req *datapb.GetIndexBuildProgressRequest)) *MockIndexCoord_GetIndexBuildProgress_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.GetIndexBuildProgressRequest))
run(args[0].(context.Context), args[1].(*datapb.GetIndexBuildProgressRequest))
})
return _c
}
func (_c *MockIndexCoord_GetIndexBuildProgress_Call) Return(_a0 *indexpb.GetIndexBuildProgressResponse, _a1 error) *MockIndexCoord_GetIndexBuildProgress_Call {
func (_c *MockIndexCoord_GetIndexBuildProgress_Call) Return(_a0 *datapb.GetIndexBuildProgressResponse, _a1 error) *MockIndexCoord_GetIndexBuildProgress_Call {
_c.Call.Return(_a0, _a1)
return _c
}
// GetIndexInfos provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
func (_m *MockIndexCoord) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
ret := _m.Called(ctx, req)
var r0 *indexpb.GetIndexInfoResponse
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.GetIndexInfoRequest) *indexpb.GetIndexInfoResponse); ok {
var r0 *datapb.GetIndexInfoResponse
if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetIndexInfoRequest) *datapb.GetIndexInfoResponse); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.GetIndexInfoResponse)
r0 = ret.Get(0).(*datapb.GetIndexInfoResponse)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.GetIndexInfoRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetIndexInfoRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -340,38 +340,38 @@ type MockIndexCoord_GetIndexInfos_Call struct {
// GetIndexInfos is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexInfoRequest
// - req *datapb.GetIndexInfoRequest
func (_e *MockIndexCoord_Expecter) GetIndexInfos(ctx interface{}, req interface{}) *MockIndexCoord_GetIndexInfos_Call {
return &MockIndexCoord_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos", ctx, req)}
}
func (_c *MockIndexCoord_GetIndexInfos_Call) Run(run func(ctx context.Context, req *indexpb.GetIndexInfoRequest)) *MockIndexCoord_GetIndexInfos_Call {
func (_c *MockIndexCoord_GetIndexInfos_Call) Run(run func(ctx context.Context, req *datapb.GetIndexInfoRequest)) *MockIndexCoord_GetIndexInfos_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.GetIndexInfoRequest))
run(args[0].(context.Context), args[1].(*datapb.GetIndexInfoRequest))
})
return _c
}
func (_c *MockIndexCoord_GetIndexInfos_Call) Return(_a0 *indexpb.GetIndexInfoResponse, _a1 error) *MockIndexCoord_GetIndexInfos_Call {
func (_c *MockIndexCoord_GetIndexInfos_Call) Return(_a0 *datapb.GetIndexInfoResponse, _a1 error) *MockIndexCoord_GetIndexInfos_Call {
_c.Call.Return(_a0, _a1)
return _c
}
// GetIndexState provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
func (_m *MockIndexCoord) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
ret := _m.Called(ctx, req)
var r0 *indexpb.GetIndexStateResponse
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.GetIndexStateRequest) *indexpb.GetIndexStateResponse); ok {
var r0 *datapb.GetIndexStateResponse
if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetIndexStateRequest) *datapb.GetIndexStateResponse); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.GetIndexStateResponse)
r0 = ret.Get(0).(*datapb.GetIndexStateResponse)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.GetIndexStateRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetIndexStateRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -387,19 +387,19 @@ type MockIndexCoord_GetIndexState_Call struct {
// GetIndexState is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexStateRequest
// - req *datapb.GetIndexStateRequest
func (_e *MockIndexCoord_Expecter) GetIndexState(ctx interface{}, req interface{}) *MockIndexCoord_GetIndexState_Call {
return &MockIndexCoord_GetIndexState_Call{Call: _e.mock.On("GetIndexState", ctx, req)}
}
func (_c *MockIndexCoord_GetIndexState_Call) Run(run func(ctx context.Context, req *indexpb.GetIndexStateRequest)) *MockIndexCoord_GetIndexState_Call {
func (_c *MockIndexCoord_GetIndexState_Call) Run(run func(ctx context.Context, req *datapb.GetIndexStateRequest)) *MockIndexCoord_GetIndexState_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.GetIndexStateRequest))
run(args[0].(context.Context), args[1].(*datapb.GetIndexStateRequest))
})
return _c
}
func (_c *MockIndexCoord_GetIndexState_Call) Return(_a0 *indexpb.GetIndexStateResponse, _a1 error) *MockIndexCoord_GetIndexState_Call {
func (_c *MockIndexCoord_GetIndexState_Call) Return(_a0 *datapb.GetIndexStateResponse, _a1 error) *MockIndexCoord_GetIndexState_Call {
_c.Call.Return(_a0, _a1)
return _c
}
@ -452,20 +452,20 @@ func (_c *MockIndexCoord_GetMetrics_Call) Return(_a0 *milvuspb.GetMetricsRespons
}
// GetSegmentIndexState provides a mock function with given fields: ctx, req
func (_m *MockIndexCoord) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
func (_m *MockIndexCoord) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
ret := _m.Called(ctx, req)
var r0 *indexpb.GetSegmentIndexStateResponse
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.GetSegmentIndexStateRequest) *indexpb.GetSegmentIndexStateResponse); ok {
var r0 *datapb.GetSegmentIndexStateResponse
if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetSegmentIndexStateRequest) *datapb.GetSegmentIndexStateResponse); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.GetSegmentIndexStateResponse)
r0 = ret.Get(0).(*datapb.GetSegmentIndexStateResponse)
}
}
var r1 error
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.GetSegmentIndexStateRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetSegmentIndexStateRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -481,19 +481,19 @@ type MockIndexCoord_GetSegmentIndexState_Call struct {
// GetSegmentIndexState is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetSegmentIndexStateRequest
// - req *datapb.GetSegmentIndexStateRequest
func (_e *MockIndexCoord_Expecter) GetSegmentIndexState(ctx interface{}, req interface{}) *MockIndexCoord_GetSegmentIndexState_Call {
return &MockIndexCoord_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState", ctx, req)}
}
func (_c *MockIndexCoord_GetSegmentIndexState_Call) Run(run func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest)) *MockIndexCoord_GetSegmentIndexState_Call {
func (_c *MockIndexCoord_GetSegmentIndexState_Call) Run(run func(ctx context.Context, req *datapb.GetSegmentIndexStateRequest)) *MockIndexCoord_GetSegmentIndexState_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.GetSegmentIndexStateRequest))
run(args[0].(context.Context), args[1].(*datapb.GetSegmentIndexStateRequest))
})
return _c
}
func (_c *MockIndexCoord_GetSegmentIndexState_Call) Return(_a0 *indexpb.GetSegmentIndexStateResponse, _a1 error) *MockIndexCoord_GetSegmentIndexState_Call {
func (_c *MockIndexCoord_GetSegmentIndexState_Call) Return(_a0 *datapb.GetSegmentIndexStateResponse, _a1 error) *MockIndexCoord_GetSegmentIndexState_Call {
_c.Call.Return(_a0, _a1)
return _c
}

View File

@ -59,9 +59,6 @@ service DataCoord {
rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {}
rpc UpdateChannelCheckpoint(UpdateChannelCheckpointRequest) returns (common.Status) {}
rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {}
rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {}
rpc SaveImportSegment(SaveImportSegmentRequest) returns(common.Status) {}
rpc UnsetIsImportingState(UnsetIsImportingStateRequest) returns(common.Status) {}
rpc MarkSegmentsDropped(MarkSegmentsDroppedRequest) returns(common.Status) {}
@ -69,6 +66,16 @@ service DataCoord {
rpc BroadcastAlteredCollection(AlterCollectionRequest) returns (common.Status) {}
rpc CheckHealth(milvus.CheckHealthRequest) returns (milvus.CheckHealthResponse) {}
rpc CreateIndex(CreateIndexRequest) returns (common.Status){}
// Deprecated: use DescribeIndex instead
rpc GetIndexState(GetIndexStateRequest) returns (GetIndexStateResponse) {}
rpc GetSegmentIndexState(GetSegmentIndexStateRequest) returns (GetSegmentIndexStateResponse) {}
rpc GetIndexInfos(GetIndexInfoRequest) returns (GetIndexInfoResponse){}
rpc DropIndex(DropIndexRequest) returns (common.Status) {}
rpc DescribeIndex(DescribeIndexRequest) returns (DescribeIndexResponse) {}
// Deprecated: use DescribeIndex instead
rpc GetIndexBuildProgress(GetIndexBuildProgressRequest) returns (GetIndexBuildProgressResponse) {}
}
service DataNode {
@ -231,6 +238,8 @@ message VchannelInfo {
repeated int64 unflushedSegmentIds = 7;
repeated int64 flushedSegmentIds = 8;
repeated int64 dropped_segmentIds = 9;
repeated int64 indexed_segmentIds = 10;
repeated SegmentInfo indexed_segments = 11;
}
message WatchDmChannelsRequest {
@ -646,3 +655,143 @@ message AlterCollectionRequest {
repeated common.KeyDataPair start_positions = 4;
repeated common.KeyValuePair properties = 5;
}
message IndexInfo {
int64 collectionID = 1;
int64 fieldID = 2;
string index_name = 3;
int64 indexID = 4;
repeated common.KeyValuePair type_params = 5;
repeated common.KeyValuePair index_params = 6;
// index build progress
// The real-time statistics may not be expected due to the existence of the compaction mechanism.
int64 indexed_rows = 7;
int64 total_rows = 8;
// index state
common.IndexState state = 9;
string index_state_fail_reason = 10;
bool is_auto_index = 11;
repeated common.KeyValuePair user_index_params = 12;
}
message FieldIndex {
IndexInfo index_info = 1;
bool deleted = 2;
uint64 create_time = 3;
}
message SegmentIndex {
int64 collectionID = 1;
int64 partitionID = 2;
int64 segmentID = 3;
int64 num_rows = 4;
int64 indexID = 5;
int64 buildID = 6;
int64 nodeID = 7;
int64 index_version = 8;
common.IndexState state = 9;
string fail_reason = 10;
repeated string index_file_keys = 11;
bool deleted = 12;
uint64 create_time = 13;
uint64 serialize_size = 14;
bool write_handoff = 15;
}
message GetIndexStateRequest {
int64 collectionID = 1;
string index_name = 2;
}
message GetIndexStateResponse {
common.Status status = 1;
common.IndexState state = 2;
string fail_reason = 3;
}
message GetSegmentIndexStateRequest {
int64 collectionID = 1;
string index_name = 2;
repeated int64 segmentIDs = 3;
}
message SegmentIndexState {
int64 segmentID = 1;
common.IndexState state = 2;
string fail_reason = 3;
}
message GetSegmentIndexStateResponse {
common.Status status = 1;
repeated SegmentIndexState states = 2;
}
message CreateIndexRequest {
int64 collectionID = 1;
int64 fieldID = 2;
string index_name = 3;
repeated common.KeyValuePair type_params = 4;
repeated common.KeyValuePair index_params = 5;
uint64 timestamp = 6;
bool is_auto_index = 7;
repeated common.KeyValuePair user_index_params = 8;
}
message GetIndexInfoRequest {
int64 collectionID = 1;
repeated int64 segmentIDs = 2;
string index_name = 3;
}
message IndexFilePathInfo {
int64 segmentID = 1;
int64 fieldID = 2;
int64 indexID = 3;
int64 buildID = 4;
string index_name = 5;
repeated common.KeyValuePair index_params = 6;
repeated string index_file_paths = 7;
uint64 serialized_size = 8;
int64 index_version = 9;
int64 num_rows = 10;
}
message SegmentIndexInfo {
int64 collectionID = 1;
int64 segmentID = 2;
bool enable_index = 3;
repeated IndexFilePathInfo index_infos = 4;
}
message GetIndexInfoResponse {
common.Status status = 1;
map<int64, SegmentIndexInfo> segment_info = 2;
}
message DropIndexRequest {
int64 collectionID = 1;
repeated int64 partitionIDs = 2;
string index_name = 3;
bool drop_all = 4;
}
message DescribeIndexRequest {
int64 collectionID = 1;
string index_name = 2;
}
message DescribeIndexResponse {
common.Status status = 1;
repeated IndexInfo index_infos = 2;
}
message GetIndexBuildProgressRequest {
int64 collectionID = 1;
string index_name = 2;
}
message GetIndexBuildProgressResponse {
common.Status status = 1;
int64 indexed_rows = 2;
int64 total_rows = 3;
}

File diff suppressed because it is too large Load Diff

View File

@ -66,21 +66,21 @@ message FieldIndex {
}
message SegmentIndex {
int64 collectionID = 1;
int64 partitionID = 2;
int64 segmentID = 3;
int64 num_rows = 4;
int64 indexID = 5;
int64 buildID = 6;
int64 nodeID = 7;
int64 index_version = 8;
common.IndexState state = 9;
string fail_reason = 10;
repeated string index_file_keys = 11;
bool deleted = 12;
uint64 create_time = 13;
uint64 serialize_size = 14;
bool write_handoff = 15;
int64 collectionID = 1;
int64 partitionID = 2;
int64 segmentID = 3;
int64 num_rows = 4;
int64 indexID = 5;
int64 buildID = 6;
int64 nodeID = 7;
int64 index_version = 8;
common.IndexState state = 9;
string fail_reason = 10;
repeated string index_file_keys = 11;
bool deleted = 12;
uint64 create_time = 13;
uint64 serialize_size = 14;
bool write_handoff = 15;
}
message RegisterNodeRequest {

View File

@ -40,6 +40,8 @@ type DataCoordMock struct {
statisticsChannel string
timeTickChannel string
checkHealthFunc func(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error)
GetIndexStateFunc func(ctx context.Context, request *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error)
DescribeIndexFunc func(ctx context.Context, request *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error)
}
func (coord *DataCoordMock) updateState(state commonpb.StateCode) {
@ -278,20 +280,70 @@ func (coord *DataCoordMock) UpdateChannelCheckpoint(ctx context.Context, req *da
}, nil
}
func (coord *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
}
func (coord *DataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
}
func (coord *DataCoordMock) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
return &datapb.GetIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
State: commonpb.IndexState_Finished,
FailReason: "",
}, nil
}
// GetSegmentIndexState gets the index state of the segments in the request from RootCoord.
func (coord *DataCoordMock) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return &datapb.GetSegmentIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
}
// GetIndexInfos gets the index files of the IndexBuildIDs in the request from RootCoordinator.
func (coord *DataCoordMock) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
return &datapb.GetIndexInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
}
// DescribeIndex describe the index info of the collection.
func (coord *DataCoordMock) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: nil,
}, nil
}
// GetIndexBuildProgress get the index building progress by num rows.
func (coord *DataCoordMock) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error) {
return &datapb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
}
func NewDataCoordMock() *DataCoordMock {
return &DataCoordMock{
nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),

View File

@ -375,7 +375,7 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol
Condition: NewTaskCondition(ctx),
LoadCollectionRequest: request,
queryCoord: node.queryCoord,
indexCoord: node.indexCoord,
datacoord: node.dataCoord,
}
log := log.Ctx(ctx).With(
@ -1120,7 +1120,7 @@ func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPar
Condition: NewTaskCondition(ctx),
LoadPartitionsRequest: request,
queryCoord: node.queryCoord,
indexCoord: node.indexCoord,
datacoord: node.dataCoord,
}
log := log.Ctx(ctx).With(
@ -1610,7 +1610,7 @@ func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateInde
Condition: NewTaskCondition(ctx),
req: request,
rootCoord: node.rootCoord,
indexCoord: node.indexCoord,
datacoord: node.dataCoord,
queryCoord: node.queryCoord,
}
@ -1689,7 +1689,7 @@ func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.Describe
ctx: ctx,
Condition: NewTaskCondition(ctx),
DescribeIndexRequest: request,
indexCoord: node.indexCoord,
datacoord: node.dataCoord,
}
method := "DescribeIndex"
@ -1774,7 +1774,7 @@ func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexReq
ctx: ctx,
Condition: NewTaskCondition(ctx),
DropIndexRequest: request,
indexCoord: node.indexCoord,
dataCoord: node.dataCoord,
queryCoord: node.queryCoord,
}
@ -1854,7 +1854,6 @@ func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb.
ctx: ctx,
Condition: NewTaskCondition(ctx),
GetIndexBuildProgressRequest: request,
indexCoord: node.indexCoord,
rootCoord: node.rootCoord,
dataCoord: node.dataCoord,
}
@ -1937,7 +1936,7 @@ func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndex
ctx: ctx,
Condition: NewTaskCondition(ctx),
GetIndexStateRequest: request,
indexCoord: node.indexCoord,
dataCoord: node.dataCoord,
rootCoord: node.rootCoord,
}
@ -4377,11 +4376,6 @@ func (node *Proxy) CheckHealth(ctx context.Context, request *milvuspb.CheckHealt
return fn("datacoord", resp, err)
})
group.Go(func() error {
resp, err := node.indexCoord.CheckHealth(ctx, request)
return fn("indexcoord", resp, err)
})
err := group.Wait()
if err != nil || len(errReasons) != 0 {
return &milvuspb.CheckHealthResponse{

View File

@ -73,7 +73,6 @@ func TestProxy_CheckHealth(t *testing.T) {
rootCoord: NewRootCoordMock(),
queryCoord: NewQueryCoordMock(),
dataCoord: NewDataCoordMock(),
indexCoord: NewIndexCoordMock(),
session: &sessionutil.Session{ServerID: 1},
}
node.multiRateLimiter = NewMultiRateLimiter()
@ -101,9 +100,6 @@ func TestProxy_CheckHealth(t *testing.T) {
dataCoordMock := NewDataCoordMock()
dataCoordMock.checkHealthFunc = checkHealthFunc1
indexCoordMock := NewIndexCoordMock()
indexCoordMock.checkHealthFunc = checkHealthFunc2
node := &Proxy{
session: &sessionutil.Session{ServerID: 1},
rootCoord: NewRootCoordMock(func(mock *RootCoordMock) {
@ -112,15 +108,14 @@ func TestProxy_CheckHealth(t *testing.T) {
queryCoord: NewQueryCoordMock(func(mock *QueryCoordMock) {
mock.checkHealthFunc = checkHealthFunc2
}),
dataCoord: dataCoordMock,
indexCoord: indexCoordMock}
dataCoord: dataCoordMock}
node.multiRateLimiter = NewMultiRateLimiter()
node.stateCode.Store(commonpb.StateCode_Healthy)
ctx := context.Background()
resp, err := node.CheckHealth(ctx, &milvuspb.CheckHealthRequest{})
assert.NoError(t, err)
assert.Equal(t, false, resp.IsHealthy)
assert.Equal(t, 4, len(resp.Reasons))
assert.Equal(t, 3, len(resp.Reasons))
})
t.Run("check quota state", func(t *testing.T) {
@ -128,7 +123,6 @@ func TestProxy_CheckHealth(t *testing.T) {
rootCoord: NewRootCoordMock(),
dataCoord: NewDataCoordMock(),
queryCoord: NewQueryCoordMock(),
indexCoord: NewIndexCoordMock(),
}
node.multiRateLimiter = NewMultiRateLimiter()
node.stateCode.Store(commonpb.StateCode_Healthy)

View File

@ -1,290 +0,0 @@
// 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 proxy
import (
"context"
"errors"
"sync/atomic"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/uniquegenerator"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type IndexCoordMock struct {
nodeID typeutil.UniqueID
address string
state atomic.Value // internal.StateCode
showConfigurationsFunc showConfigurationsFuncType
getMetricsFunc getMetricsFuncType
statisticsChannel string
timeTickChannel string
minioBucketName string
checkHealthFunc func(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error)
}
func (coord *IndexCoordMock) updateState(state commonpb.StateCode) {
coord.state.Store(state)
}
func (coord *IndexCoordMock) getState() commonpb.StateCode {
return coord.state.Load().(commonpb.StateCode)
}
func (coord *IndexCoordMock) healthy() bool {
return coord.getState() == commonpb.StateCode_Healthy
}
func (coord *IndexCoordMock) Init() error {
coord.updateState(commonpb.StateCode_Initializing)
return nil
}
func (coord *IndexCoordMock) Start() error {
defer coord.updateState(commonpb.StateCode_Healthy)
return nil
}
func (coord *IndexCoordMock) Stop() error {
defer coord.updateState(commonpb.StateCode_Abnormal)
return nil
}
func (coord *IndexCoordMock) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: coord.nodeID,
Role: typeutil.IndexCoordRole,
StateCode: coord.getState(),
ExtraInfo: nil,
},
SubcomponentStates: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
}
func (coord *IndexCoordMock) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Value: coord.statisticsChannel,
}, nil
}
func (coord *IndexCoordMock) Register() error {
return nil
}
func (coord *IndexCoordMock) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Value: coord.timeTickChannel,
}, nil
}
func (coord *IndexCoordMock) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
}
func (coord *IndexCoordMock) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
}, nil
}
func (coord *IndexCoordMock) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return &indexpb.GetIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
State: commonpb.IndexState_Finished,
FailReason: "",
}, nil
}
// GetSegmentIndexState gets the index state of the segments in the request from RootCoord.
func (coord *IndexCoordMock) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return &indexpb.GetSegmentIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
}
// GetIndexInfos gets the index files of the IndexBuildIDs in the request from RootCoordinator.
func (coord *IndexCoordMock) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
return &indexpb.GetIndexInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
}
// DescribeIndex describe the index info of the collection.
func (coord *IndexCoordMock) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: nil,
}, nil
}
// GetIndexBuildProgress get the index building progress by num rows.
func (coord *IndexCoordMock) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return &indexpb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
}
func (coord *IndexCoordMock) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
if !coord.healthy() {
return &internalpb.ShowConfigurationsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "unhealthy",
},
}, nil
}
if coord.showConfigurationsFunc != nil {
return coord.showConfigurationsFunc(ctx, req)
}
return &internalpb.ShowConfigurationsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "not implemented",
},
}, nil
}
func (coord *IndexCoordMock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
if !coord.healthy() {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "unhealthy",
},
}, nil
}
if coord.getMetricsFunc != nil {
return coord.getMetricsFunc(ctx, req)
}
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "not implemented",
},
Response: "",
ComponentName: "",
}, nil
}
func (coord *IndexCoordMock) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
if coord.checkHealthFunc != nil {
return coord.checkHealthFunc(ctx, req)
}
return &milvuspb.CheckHealthResponse{IsHealthy: true}, nil
}
func NewIndexCoordMock() *IndexCoordMock {
return &IndexCoordMock{
nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
address: funcutil.GenRandomStr(), // TODO(dragondriver): random address
statisticsChannel: funcutil.GenRandomStr(),
timeTickChannel: funcutil.GenRandomStr(),
minioBucketName: funcutil.GenRandomStr(),
}
}
type GetIndexStateFunc func(ctx context.Context, request *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error)
type DescribeIndexFunc func(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error)
type DropIndexFunc func(ctx context.Context, request *indexpb.DropIndexRequest) (*commonpb.Status, error)
type mockIndexCoord struct {
types.IndexCoord
GetIndexStateFunc
DescribeIndexFunc
DropIndexFunc
}
func (m *mockIndexCoord) GetIndexState(ctx context.Context, request *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
if m.GetIndexStateFunc != nil {
log.Warn("func not nil")
return m.GetIndexStateFunc(ctx, request)
}
log.Warn("func nil")
return nil, errors.New("mock")
}
func (m *mockIndexCoord) DescribeIndex(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
if m.DescribeIndexFunc != nil {
log.Warn("DescribeIndexFunc not nil")
return m.DescribeIndexFunc(ctx, request)
}
log.Warn("DescribeIndexFunc nil")
return nil, errors.New("mock")
}
func (m *mockIndexCoord) DropIndex(ctx context.Context, request *indexpb.DropIndexRequest) (*commonpb.Status, error) {
if m.DropIndexFunc != nil {
log.Warn("DropIndexFunc not nil")
return m.DropIndexFunc(ctx, request)
}
log.Warn("DropIndexFunc is nil")
return nil, errors.New("mock")
}
func newMockIndexCoord() *mockIndexCoord {
return &mockIndexCoord{}
}

View File

@ -201,15 +201,6 @@ func getSystemInfoMetrics(
dataCoordErr = metricsinfo.UnmarshalTopology(dataCoordResp.Response, &dataCoordTopology)
}()
wg.Add(1)
go func() {
defer wg.Done()
indexCoordResp, indexCoordErr = node.indexCoord.GetMetrics(ctx, request)
indexCoordRoleName = indexCoordResp.GetComponentName()
indexCoordErr = metricsinfo.UnmarshalTopology(indexCoordResp.Response, &indexCoordTopology)
}()
wg.Add(1)
go func() {
defer wg.Done()

View File

@ -51,15 +51,10 @@ func TestProxy_metrics(t *testing.T) {
dc.Start()
defer dc.Stop()
ic := NewIndexCoordMock()
ic.Start()
defer ic.Stop()
proxy := &Proxy{
rootCoord: rc,
queryCoord: qc,
dataCoord: dc,
indexCoord: ic,
session: &sessionutil.Session{Address: funcutil.GenRandomStr()},
}
@ -221,63 +216,6 @@ func TestProxy_metrics(t *testing.T) {
}
ic.getMetricsFunc = func(ctx context.Context, request *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
id := typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
clusterTopology := metricsinfo.IndexClusterTopology{
Self: metricsinfo.IndexCoordInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
Name: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, id),
HardwareInfos: metricsinfo.HardwareMetrics{},
SystemInfo: metricsinfo.DeployMetrics{},
Type: typeutil.IndexCoordRole,
ID: id,
},
SystemConfigurations: metricsinfo.IndexCoordConfiguration{},
},
ConnectedNodes: make([]metricsinfo.IndexNodeInfos, 0),
}
infos := metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{},
SystemConfigurations: metricsinfo.IndexNodeConfiguration{},
}
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, infos)
coordTopology := metricsinfo.IndexCoordTopology{
Cluster: clusterTopology,
Connections: metricsinfo.ConnTopology{
Name: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, id),
ConnectedComponents: []metricsinfo.ConnectionInfo{
{
TargetName: metricsinfo.ConstructComponentName(typeutil.RootCoordRole, id),
TargetType: typeutil.RootCoordRole,
},
{
TargetName: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, id),
TargetType: typeutil.QueryCoordRole,
},
{
TargetName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, id),
TargetType: typeutil.DataCoordRole,
},
},
},
}
resp, _ := metricsinfo.MarshalTopology(coordTopology)
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Response: resp,
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, id),
}, nil
}
req, _ := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
resp, err := getSystemInfoMetrics(ctx, req, proxy)
assert.NoError(t, err)
@ -286,5 +224,4 @@ func TestProxy_metrics(t *testing.T) {
rc.getMetricsFunc = nil
qc.getMetricsFunc = nil
dc.getMetricsFunc = nil
ic.getMetricsFunc = nil
}

View File

@ -81,7 +81,6 @@ type Proxy struct {
etcdCli *clientv3.Client
address string
rootCoord types.RootCoord
indexCoord types.IndexCoord
dataCoord types.DataCoord
queryCoord types.QueryCoord
@ -449,11 +448,6 @@ func (node *Proxy) SetRootCoordClient(cli types.RootCoord) {
node.rootCoord = cli
}
// SetIndexCoordClient sets IndexCoord client for proxy.
func (node *Proxy) SetIndexCoordClient(cli types.IndexCoord) {
node.indexCoord = cli
}
// SetDataCoordClient sets DataCoord client for proxy.
func (node *Proxy) SetDataCoordClient(cli types.DataCoord) {
node.dataCoord = cli

View File

@ -39,15 +39,12 @@ import (
grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord"
grpcdatacoordclient2 "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode"
grpcindexcoord "github.com/milvus-io/milvus/internal/distributed/indexcoord"
grpcindexcoordclient "github.com/milvus-io/milvus/internal/distributed/indexcoord/client"
grpcindexnode "github.com/milvus-io/milvus/internal/distributed/indexnode"
grpcquerycoord "github.com/milvus-io/milvus/internal/distributed/querycoord"
grpcquerycoordclient "github.com/milvus-io/milvus/internal/distributed/querycoord/client"
grpcquerynode "github.com/milvus-io/milvus/internal/distributed/querynode"
grpcrootcoord "github.com/milvus-io/milvus/internal/distributed/rootcoord"
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
"github.com/milvus-io/milvus/internal/indexcoord"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
@ -232,35 +229,6 @@ func runDataNode(ctx context.Context, localMsg bool, alias string) *grpcdatanode
return dn
}
func runIndexCoord(ctx context.Context, localMsg bool) *grpcindexcoord.Server {
var is *grpcindexcoord.Server
var wg sync.WaitGroup
wg.Add(1)
go func() {
if !localMsg {
logutil.SetupLogger(&indexcoord.Params.Log)
defer log.Sync()
}
factory := dependency.NewDefaultFactory(localMsg)
var err error
is, err = grpcindexcoord.NewServer(ctx, factory)
if err != nil {
panic(err)
}
wg.Done()
err = is.Run()
if err != nil {
panic(err)
}
}()
wg.Wait()
metrics.RegisterIndexCoord(Registry)
return is
}
func runIndexNode(ctx context.Context, localMsg bool, alias string) *grpcindexnode.Server {
var in *grpcindexnode.Server
var wg sync.WaitGroup
@ -468,17 +436,6 @@ func TestProxy(t *testing.T) {
}()
}
ic := runIndexCoord(ctx, localMsg)
log.Info("running IndexCoord ...")
if ic != nil {
defer func() {
err := ic.Stop()
assert.NoError(t, err)
log.Info("stop IndexCoord")
}()
}
in := runIndexNode(ctx, localMsg, alias)
log.Info("running IndexNode ...")
@ -547,15 +504,6 @@ func TestProxy(t *testing.T) {
proxy.SetQueryCoordClient(queryCoordClient)
log.Info("Proxy set query coordinator client")
indexCoordClient, err := grpcindexcoordclient.NewClient(ctx, Params.EtcdCfg.MetaRootPath.GetValue(), etcdcli)
assert.NoError(t, err)
err = indexCoordClient.Init()
assert.NoError(t, err)
err = funcutil.WaitForComponentHealthy(ctx, indexCoordClient, typeutil.IndexCoordRole, attempts, sleepDuration)
assert.NoError(t, err)
proxy.SetIndexCoordClient(indexCoordClient)
log.Info("Proxy set index coordinator client")
proxy.UpdateStateCode(commonpb.StateCode_Initializing)
err = proxy.Init()
assert.NoError(t, err)

View File

@ -32,7 +32,6 @@ import (
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
@ -1243,7 +1242,7 @@ type loadCollectionTask struct {
*milvuspb.LoadCollectionRequest
ctx context.Context
queryCoord types.QueryCoord
indexCoord types.IndexCoord
datacoord types.DataCoord
result *commonpb.Status
collectionID UniqueID
@ -1324,7 +1323,7 @@ func (lct *loadCollectionTask) Execute(ctx context.Context) (err error) {
return err
}
// check index
indexResponse, err := lct.indexCoord.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{
indexResponse, err := lct.datacoord.DescribeIndex(ctx, &datapb.DescribeIndexRequest{
CollectionID: collID,
IndexName: "",
})
@ -1474,7 +1473,7 @@ type loadPartitionsTask struct {
*milvuspb.LoadPartitionsRequest
ctx context.Context
queryCoord types.QueryCoord
indexCoord types.IndexCoord
datacoord types.DataCoord
result *commonpb.Status
collectionID UniqueID
@ -1542,7 +1541,7 @@ func (lpt *loadPartitionsTask) Execute(ctx context.Context) error {
return err
}
// check index
indexResponse, err := lpt.indexCoord.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{
indexResponse, err := lpt.datacoord.DescribeIndex(ctx, &datapb.DescribeIndexRequest{
CollectionID: collID,
IndexName: "",
})

View File

@ -22,6 +22,8 @@ import (
"fmt"
"strconv"
"github.com/milvus-io/milvus/internal/proto/datapb"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -29,7 +31,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
@ -55,7 +56,7 @@ type createIndexTask struct {
req *milvuspb.CreateIndexRequest
ctx context.Context
rootCoord types.RootCoord
indexCoord types.IndexCoord
datacoord types.DataCoord
queryCoord types.QueryCoord
result *commonpb.Status
@ -290,14 +291,6 @@ func (cit *createIndexTask) PreExecute(ctx context.Context) error {
return err
}
loaded, err := isCollectionLoaded(ctx, cit.queryCoord, collID)
if err != nil {
return err
}
if loaded {
return fmt.Errorf("create index failed, collection is loaded, please release it first")
}
return nil
}
@ -310,7 +303,7 @@ func (cit *createIndexTask) Execute(ctx context.Context) error {
cit.req.IndexName = Params.CommonCfg.DefaultIndexName.GetValue() + "_" + strconv.FormatInt(cit.fieldSchema.GetFieldID(), 10)
}
var err error
req := &indexpb.CreateIndexRequest{
req := &datapb.CreateIndexRequest{
CollectionID: cit.collectionID,
FieldID: cit.fieldSchema.GetFieldID(),
IndexName: cit.req.GetIndexName(),
@ -320,7 +313,7 @@ func (cit *createIndexTask) Execute(ctx context.Context) error {
UserIndexParams: cit.req.GetExtraParams(),
Timestamp: cit.BeginTs(),
}
cit.result, err = cit.indexCoord.CreateIndex(ctx, req)
cit.result, err = cit.datacoord.CreateIndex(ctx, req)
if err != nil {
return err
}
@ -337,9 +330,9 @@ func (cit *createIndexTask) PostExecute(ctx context.Context) error {
type describeIndexTask struct {
Condition
*milvuspb.DescribeIndexRequest
ctx context.Context
indexCoord types.IndexCoord
result *milvuspb.DescribeIndexResponse
ctx context.Context
datacoord types.DataCoord
result *milvuspb.DescribeIndexResponse
collectionID UniqueID
}
@ -409,7 +402,7 @@ func (dit *describeIndexTask) Execute(ctx context.Context) error {
return fmt.Errorf("failed to parse collection schema: %s", err)
}
resp, err := dit.indexCoord.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{CollectionID: dit.collectionID, IndexName: dit.IndexName})
resp, err := dit.datacoord.DescribeIndex(ctx, &datapb.DescribeIndexRequest{CollectionID: dit.collectionID, IndexName: dit.IndexName})
if err != nil || resp == nil {
return err
}
@ -451,7 +444,7 @@ type dropIndexTask struct {
Condition
ctx context.Context
*milvuspb.DropIndexRequest
indexCoord types.IndexCoord
dataCoord types.DataCoord
queryCoord types.QueryCoord
result *commonpb.Status
@ -531,7 +524,7 @@ func (dit *dropIndexTask) PreExecute(ctx context.Context) error {
func (dit *dropIndexTask) Execute(ctx context.Context) error {
var err error
dit.result, err = dit.indexCoord.DropIndex(ctx, &indexpb.DropIndexRequest{
dit.result, err = dit.dataCoord.DropIndex(ctx, &datapb.DropIndexRequest{
CollectionID: dit.collectionID,
PartitionIDs: nil,
IndexName: dit.IndexName,
@ -554,11 +547,10 @@ func (dit *dropIndexTask) PostExecute(ctx context.Context) error {
type getIndexBuildProgressTask struct {
Condition
*milvuspb.GetIndexBuildProgressRequest
ctx context.Context
indexCoord types.IndexCoord
rootCoord types.RootCoord
dataCoord types.DataCoord
result *milvuspb.GetIndexBuildProgressResponse
ctx context.Context
rootCoord types.RootCoord
dataCoord types.DataCoord
result *milvuspb.GetIndexBuildProgressResponse
collectionID UniqueID
}
@ -623,7 +615,7 @@ func (gibpt *getIndexBuildProgressTask) Execute(ctx context.Context) error {
gibpt.IndexName = Params.CommonCfg.DefaultIndexName.GetValue()
}
resp, err := gibpt.indexCoord.GetIndexBuildProgress(ctx, &indexpb.GetIndexBuildProgressRequest{
resp, err := gibpt.dataCoord.GetIndexBuildProgress(ctx, &datapb.GetIndexBuildProgressRequest{
CollectionID: collectionID,
IndexName: gibpt.IndexName,
})
@ -648,10 +640,10 @@ func (gibpt *getIndexBuildProgressTask) PostExecute(ctx context.Context) error {
type getIndexStateTask struct {
Condition
*milvuspb.GetIndexStateRequest
ctx context.Context
indexCoord types.IndexCoord
rootCoord types.RootCoord
result *milvuspb.GetIndexStateResponse
ctx context.Context
dataCoord types.DataCoord
rootCoord types.RootCoord
result *milvuspb.GetIndexStateResponse
collectionID UniqueID
}
@ -714,7 +706,7 @@ func (gist *getIndexStateTask) Execute(ctx context.Context) error {
return err
}
state, err := gist.indexCoord.GetIndexState(ctx, &indexpb.GetIndexStateRequest{
state, err := gist.dataCoord.GetIndexState(ctx, &datapb.GetIndexStateRequest{
CollectionID: collectionID,
IndexName: gist.IndexName,
})

View File

@ -27,7 +27,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
@ -49,8 +49,8 @@ func TestGetIndexStateTask_Execute(t *testing.T) {
ctx := context.Background()
rootCoord := newMockRootCoord()
indexCoord := newMockIndexCoord()
queryCoord := NewQueryCoordMock()
datacoord := NewDataCoordMock()
gist := &getIndexStateTask{
GetIndexStateRequest: &milvuspb.GetIndexStateRequest{
@ -60,9 +60,9 @@ func TestGetIndexStateTask_Execute(t *testing.T) {
FieldName: fieldName,
IndexName: indexName,
},
ctx: ctx,
indexCoord: indexCoord,
rootCoord: rootCoord,
ctx: ctx,
rootCoord: rootCoord,
dataCoord: datacoord,
result: &milvuspb.GetIndexStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: "mock"},
State: commonpb.IndexState_Unissued,
@ -86,8 +86,8 @@ func TestGetIndexStateTask_Execute(t *testing.T) {
}, nil
}
indexCoord.GetIndexStateFunc = func(ctx context.Context, request *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return &indexpb.GetIndexStateResponse{
datacoord.GetIndexStateFunc = func(ctx context.Context, request *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error) {
return &datapb.GetIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
@ -116,7 +116,7 @@ func TestDropIndexTask_PreExecute(t *testing.T) {
}, nil
}
qc := NewQueryCoordMock(withValidShardLeaders(), SetQueryCoordShowCollectionsFunc(showCollectionMock))
ic := newMockIndexCoord()
dc := NewDataCoordMock()
ctx := context.Background()
qc.updateState(commonpb.StateCode_Healthy)
@ -140,7 +140,7 @@ func TestDropIndexTask_PreExecute(t *testing.T) {
FieldName: fieldName,
IndexName: indexName,
},
indexCoord: ic,
dataCoord: dc,
queryCoord: qc,
result: nil,
collectionID: collectionID,
@ -218,7 +218,7 @@ func TestCreateIndexTask_PreExecute(t *testing.T) {
collectionID := UniqueID(1)
fieldName := newTestSchema().Fields[0].Name
ic := newMockIndexCoord()
dc := NewDataCoordMock()
ctx := context.Background()
mockCache := newMockCache()
@ -239,7 +239,7 @@ func TestCreateIndexTask_PreExecute(t *testing.T) {
CollectionName: collectionName,
FieldName: fieldName,
},
indexCoord: ic,
datacoord: dc,
queryCoord: nil,
result: nil,
collectionID: collectionID,
@ -261,37 +261,4 @@ func TestCreateIndexTask_PreExecute(t *testing.T) {
err := cit.PreExecute(ctx)
assert.NoError(t, err)
})
t.Run("coll has been loaded", func(t *testing.T) {
showCollectionMock := func(ctx context.Context, request *querypb.ShowCollectionsRequest) (*querypb.ShowCollectionsResponse, error) {
return &querypb.ShowCollectionsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
CollectionIDs: []int64{collectionID},
}, nil
}
qc := NewQueryCoordMock(withValidShardLeaders(), SetQueryCoordShowCollectionsFunc(showCollectionMock))
qc.updateState(commonpb.StateCode_Healthy)
cit.queryCoord = qc
err := cit.PreExecute(ctx)
assert.Error(t, err)
})
t.Run("check load error", func(t *testing.T) {
showCollectionMock := func(ctx context.Context, request *querypb.ShowCollectionsRequest) (*querypb.ShowCollectionsResponse, error) {
return &querypb.ShowCollectionsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
CollectionIDs: nil,
}, errors.New("error")
}
qc := NewQueryCoordMock(withValidShardLeaders(), SetQueryCoordShowCollectionsFunc(showCollectionMock))
qc.updateState(commonpb.StateCode_Healthy)
cit.queryCoord = qc
err := cit.PreExecute(ctx)
assert.Error(t, err)
})
}

View File

@ -27,31 +27,24 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/distance"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/internal/util/uniquegenerator"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
)
// TODO(dragondriver): add more test cases
@ -2354,7 +2347,7 @@ func Test_dropCollectionTask_PostExecute(t *testing.T) {
func Test_loadCollectionTask_Execute(t *testing.T) {
rc := newMockRootCoord()
qc := NewQueryCoordMock(withValidShardLeaders())
ic := newMockIndexCoord()
dc := NewDataCoordMock()
dbName := funcutil.GenRandomStr()
collectionName := funcutil.GenRandomStr()
@ -2394,7 +2387,7 @@ func Test_loadCollectionTask_Execute(t *testing.T) {
},
ctx: ctx,
queryCoord: qc,
indexCoord: ic,
datacoord: dc,
result: nil,
collectionID: 0,
}
@ -2405,8 +2398,8 @@ func Test_loadCollectionTask_Execute(t *testing.T) {
})
t.Run("indexcoord describe index not success", func(t *testing.T) {
ic.DescribeIndexFunc = func(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
dc.DescribeIndexFunc = func(ctx context.Context, request *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
@ -2419,12 +2412,12 @@ func Test_loadCollectionTask_Execute(t *testing.T) {
})
t.Run("no vector index", func(t *testing.T) {
ic.DescribeIndexFunc = func(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
dc.DescribeIndexFunc = func(ctx context.Context, request *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
IndexInfos: []*datapb.IndexInfo{
{
CollectionID: collectionID,
FieldID: 100,
@ -2451,7 +2444,7 @@ func Test_loadCollectionTask_Execute(t *testing.T) {
func Test_loadPartitionTask_Execute(t *testing.T) {
rc := newMockRootCoord()
qc := NewQueryCoordMock(withValidShardLeaders())
ic := newMockIndexCoord()
dc := NewDataCoordMock()
dbName := funcutil.GenRandomStr()
collectionName := funcutil.GenRandomStr()
@ -2491,7 +2484,7 @@ func Test_loadPartitionTask_Execute(t *testing.T) {
},
ctx: ctx,
queryCoord: qc,
indexCoord: ic,
datacoord: dc,
result: nil,
collectionID: 0,
}
@ -2502,8 +2495,8 @@ func Test_loadPartitionTask_Execute(t *testing.T) {
})
t.Run("indexcoord describe index not success", func(t *testing.T) {
ic.DescribeIndexFunc = func(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
dc.DescribeIndexFunc = func(ctx context.Context, request *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
@ -2516,12 +2509,12 @@ func Test_loadPartitionTask_Execute(t *testing.T) {
})
t.Run("no vector index", func(t *testing.T) {
ic.DescribeIndexFunc = func(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
dc.DescribeIndexFunc = func(ctx context.Context, request *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
IndexInfos: []*datapb.IndexInfo{
{
CollectionID: collectionID,
FieldID: 100,

View File

@ -27,7 +27,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/types"
@ -167,7 +166,7 @@ func (broker *CoordinatorBroker) GetIndexInfo(ctx context.Context, collectionID
ctx, cancel := context.WithTimeout(ctx, brokerRPCTimeout)
defer cancel()
resp, err := broker.indexCoord.GetIndexInfos(ctx, &indexpb.GetIndexInfoRequest{
resp, err := broker.dataCoord.GetIndexInfos(ctx, &datapb.GetIndexInfoRequest{
CollectionID: collectionID,
SegmentIDs: []int64{segmentID},
})

View File

@ -10,7 +10,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -38,8 +37,6 @@ type Broker interface {
WatchChannels(ctx context.Context, info *watchInfo) error
UnwatchChannels(ctx context.Context, info *watchInfo) error
AddSegRefLock(ctx context.Context, taskID int64, segIDs []int64) error
ReleaseSegRefLock(ctx context.Context, taskID int64, segIDs []int64) error
Flush(ctx context.Context, cID int64, segIDs []int64) error
Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
UnsetIsImportingState(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
@ -47,8 +44,8 @@ type Broker interface {
GetSegmentStates(context.Context, *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error)
DropCollectionIndex(ctx context.Context, collID UniqueID, partIDs []UniqueID) error
GetSegmentIndexState(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error)
DescribeIndex(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error)
GetSegmentIndexState(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error)
DescribeIndex(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error)
BroadcastAlteredCollection(ctx context.Context, req *milvuspb.AlterCollectionRequest) error
}
@ -130,48 +127,6 @@ func (b *ServerBroker) UnwatchChannels(ctx context.Context, info *watchInfo) err
return nil
}
func (b *ServerBroker) AddSegRefLock(ctx context.Context, taskID int64, segIDs []int64) error {
log.Info("acquiring seg lock",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", b.s.session.ServerID))
resp, err := b.s.dataCoord.AcquireSegmentLock(ctx, &datapb.AcquireSegmentLockRequest{
SegmentIDs: segIDs,
NodeID: b.s.session.ServerID,
TaskID: taskID,
})
if err != nil {
return err
}
if resp.GetErrorCode() != commonpb.ErrorCode_Success {
return fmt.Errorf("failed to acquire segment lock %s", resp.GetReason())
}
log.Info("acquire seg lock succeed",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", b.s.session.ServerID))
return nil
}
func (b *ServerBroker) ReleaseSegRefLock(ctx context.Context, taskID int64, segIDs []int64) error {
log.Info("releasing seg lock",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", b.s.session.ServerID))
resp, err := b.s.dataCoord.ReleaseSegmentLock(ctx, &datapb.ReleaseSegmentLockRequest{
SegmentIDs: segIDs,
NodeID: b.s.session.ServerID,
TaskID: taskID,
})
if err != nil {
return err
}
if resp.GetErrorCode() != commonpb.ErrorCode_Success {
return fmt.Errorf("failed to release segment lock %s", resp.GetReason())
}
log.Info("release seg lock succeed",
zap.Int64s("segment IDs", segIDs),
zap.Int64("node ID", b.s.session.ServerID))
return nil
}
func (b *ServerBroker) Flush(ctx context.Context, cID int64, segIDs []int64) error {
resp, err := b.s.dataCoord.Flush(ctx, &datapb.FlushRequest{
Base: commonpbutil.NewMsgBase(
@ -209,7 +164,7 @@ func (b *ServerBroker) GetSegmentStates(ctx context.Context, req *datapb.GetSegm
}
func (b *ServerBroker) DropCollectionIndex(ctx context.Context, collID UniqueID, partIDs []UniqueID) error {
rsp, err := b.s.indexCoord.DropIndex(ctx, &indexpb.DropIndexRequest{
rsp, err := b.s.dataCoord.DropIndex(ctx, &datapb.DropIndexRequest{
CollectionID: collID,
PartitionIDs: partIDs,
IndexName: "",
@ -224,8 +179,8 @@ func (b *ServerBroker) DropCollectionIndex(ctx context.Context, collID UniqueID,
return nil
}
func (b *ServerBroker) GetSegmentIndexState(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
resp, err := b.s.indexCoord.GetSegmentIndexState(ctx, &indexpb.GetSegmentIndexStateRequest{
func (b *ServerBroker) GetSegmentIndexState(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
resp, err := b.s.dataCoord.GetSegmentIndexState(ctx, &datapb.GetSegmentIndexStateRequest{
CollectionID: collID,
IndexName: indexName,
SegmentIDs: segIDs,
@ -277,8 +232,8 @@ func (b *ServerBroker) BroadcastAlteredCollection(ctx context.Context, req *milv
return nil
}
func (b *ServerBroker) DescribeIndex(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return b.s.indexCoord.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{
func (b *ServerBroker) DescribeIndex(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return b.s.dataCoord.DescribeIndex(ctx, &datapb.DescribeIndexRequest{
CollectionID: colID,
})
}

View File

@ -9,8 +9,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/stretchr/testify/assert"
@ -109,58 +107,6 @@ func TestServerBroker_UnwatchChannels(t *testing.T) {
b.UnwatchChannels(ctx, &watchInfo{})
}
func TestServerBroker_AddSegRefLock(t *testing.T) {
t.Run("failed to execute", func(t *testing.T) {
c := newTestCore(withInvalidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.AddSegRefLock(ctx, 1, []int64{1, 2})
assert.Error(t, err)
})
t.Run("non success error code on execute", func(t *testing.T) {
c := newTestCore(withFailedDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.AddSegRefLock(ctx, 1, []int64{1, 2})
assert.Error(t, err)
})
t.Run("success", func(t *testing.T) {
c := newTestCore(withValidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.AddSegRefLock(ctx, 1, []int64{1, 2})
assert.NoError(t, err)
})
}
func TestServerBroker_ReleaseSegRefLock(t *testing.T) {
t.Run("failed to execute", func(t *testing.T) {
c := newTestCore(withInvalidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.ReleaseSegRefLock(ctx, 1, []int64{1, 2})
assert.Error(t, err)
})
t.Run("non success error code on execute", func(t *testing.T) {
c := newTestCore(withFailedDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.ReleaseSegRefLock(ctx, 1, []int64{1, 2})
assert.Error(t, err)
})
t.Run("success", func(t *testing.T) {
c := newTestCore(withValidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.ReleaseSegRefLock(ctx, 1, []int64{1, 2})
assert.NoError(t, err)
})
}
func TestServerBroker_Flush(t *testing.T) {
t.Run("failed to execute", func(t *testing.T) {
c := newTestCore(withInvalidDataCoord())
@ -218,7 +164,7 @@ func TestServerBroker_Import(t *testing.T) {
func TestServerBroker_DropCollectionIndex(t *testing.T) {
t.Run("failed to execute", func(t *testing.T) {
c := newTestCore(withInvalidIndexCoord())
c := newTestCore(withInvalidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.DropCollectionIndex(ctx, 1, nil)
@ -226,7 +172,7 @@ func TestServerBroker_DropCollectionIndex(t *testing.T) {
})
t.Run("non success error code on execute", func(t *testing.T) {
c := newTestCore(withFailedIndexCoord())
c := newTestCore(withFailedDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.DropCollectionIndex(ctx, 1, nil)
@ -234,7 +180,7 @@ func TestServerBroker_DropCollectionIndex(t *testing.T) {
})
t.Run("success", func(t *testing.T) {
c := newTestCore(withValidIndexCoord())
c := newTestCore(withValidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
err := b.DropCollectionIndex(ctx, 1, nil)
@ -244,7 +190,7 @@ func TestServerBroker_DropCollectionIndex(t *testing.T) {
func TestServerBroker_GetSegmentIndexState(t *testing.T) {
t.Run("failed to execute", func(t *testing.T) {
c := newTestCore(withInvalidIndexCoord())
c := newTestCore(withInvalidDataCoord())
b := newServerBroker(c)
ctx := context.Background()
_, err := b.GetSegmentIndexState(ctx, 1, "index_name", []UniqueID{1, 2})
@ -252,7 +198,7 @@ func TestServerBroker_GetSegmentIndexState(t *testing.T) {
})
t.Run("non success error code on execute", func(t *testing.T) {
c := newTestCore(withFailedIndexCoord())
c := newTestCore(withFailedDataCoord())
b := newServerBroker(c)
ctx := context.Background()
_, err := b.GetSegmentIndexState(ctx, 1, "index_name", []UniqueID{1, 2})
@ -260,11 +206,11 @@ func TestServerBroker_GetSegmentIndexState(t *testing.T) {
})
t.Run("success", func(t *testing.T) {
c := newTestCore(withValidIndexCoord())
c.indexCoord.(*mockIndexCoord).GetSegmentIndexStateFunc = func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return &indexpb.GetSegmentIndexStateResponse{
c := newTestCore(withValidDataCoord())
c.dataCoord.(*mockDataCoord).GetSegmentIndexStateFunc = func(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return &datapb.GetSegmentIndexStateResponse{
Status: succStatus(),
States: []*indexpb.SegmentIndexState{
States: []*datapb.SegmentIndexState{
{
SegmentID: 1,
State: commonpb.IndexState_Finished,

View File

@ -69,12 +69,6 @@ func (t *dropPartitionTask) Execute(ctx context.Context) error {
ts: t.GetTs(),
})
redoTask.AddAsyncStep(&dropIndexStep{
baseStep: baseStep{core: t.core},
collID: t.collMeta.CollectionID,
partIDs: []UniqueID{partID},
})
// TODO: release partition when query coord is ready.
redoTask.AddAsyncStep(&deletePartitionDataStep{
baseStep: baseStep{core: t.core},

View File

@ -118,11 +118,6 @@ func (c *bgGarbageCollector) ReDropPartition(pChannels []string, partition *mode
baseStep: baseStep{core: c.s},
pChannels: pChannels,
})
redo.AddAsyncStep(&dropIndexStep{
baseStep: baseStep{core: c.s},
collID: partition.CollectionID,
partIDs: []UniqueID{partition.PartitionID},
})
redo.AddAsyncStep(&removePartitionMetaStep{
baseStep: baseStep{core: c.s},
collectionID: partition.CollectionID,

View File

@ -6,7 +6,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap"
)
@ -16,8 +15,8 @@ type IDAllocator func(count uint32) (UniqueID, UniqueID, error)
type ImportFunc func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
type MarkSegmentsDroppedFunc func(ctx context.Context, segIDs []int64) (*commonpb.Status, error)
type GetSegmentStatesFunc func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error)
type DescribeIndexFunc func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error)
type GetSegmentIndexStateFunc func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error)
type DescribeIndexFunc func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error)
type GetSegmentIndexStateFunc func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error)
type UnsetIsImportingStateFunc func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
type ImportFactory interface {
@ -115,13 +114,13 @@ func GetSegmentStatesWithCore(c *Core) GetSegmentStatesFunc {
}
func DescribeIndexWithCore(c *Core) DescribeIndexFunc {
return func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return c.broker.DescribeIndex(ctx, colID)
}
}
func GetSegmentIndexStateWithCore(c *Core) GetSegmentIndexStateFunc {
return func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
return func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
return c.broker.GetSegmentIndexState(ctx, collID, indexName, segIDs)
}
}

View File

@ -32,7 +32,6 @@ import (
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/importutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
@ -86,8 +85,8 @@ type importManager struct {
getCollectionName func(collID, partitionID typeutil.UniqueID) (string, string, error)
callMarkSegmentsDropped func(ctx context.Context, segIDs []typeutil.UniqueID) (*commonpb.Status, error)
callGetSegmentStates func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error)
callDescribeIndex func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error)
callGetSegmentIndexState func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error)
callDescribeIndex func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error)
callGetSegmentIndexState func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error)
callUnsetIsImportingState func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
}
@ -98,8 +97,8 @@ func newImportManager(ctx context.Context, client kv.TxnKV,
markSegmentsDropped func(ctx context.Context, segIDs []typeutil.UniqueID) (*commonpb.Status, error),
getSegmentStates func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error),
getCollectionName func(collID, partitionID typeutil.UniqueID) (string, string, error),
describeIndex func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error),
getSegmentIndexState func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error),
describeIndex func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error),
getSegmentIndexState func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error),
unsetIsImportingState func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)) *importManager {
mgr := &importManager{
ctx: ctx,
@ -456,7 +455,7 @@ func (m *importManager) checkIndexingDone(ctx context.Context, collID UniqueID,
}
// Check if collection has indexed fields.
var descIdxResp *indexpb.DescribeIndexResponse
var descIdxResp *datapb.DescribeIndexResponse
var err error
if descIdxResp, err = m.callDescribeIndex(ctx, collID); err != nil {
log.Error("failed to describe index",

View File

@ -30,7 +30,6 @@ import (
memkv "github.com/milvus-io/milvus/internal/kv/mem"
"github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/importutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
@ -469,6 +468,7 @@ func TestImportManager_TestFlipTaskStateLoop(t *testing.T) {
ErrorCode: commonpb.ErrorCode_Success,
}, nil
}
callGetSegmentStates := func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
return &datapb.GetSegmentStatesResponse{
Status: &commonpb.Status{
@ -476,19 +476,20 @@ func TestImportManager_TestFlipTaskStateLoop(t *testing.T) {
},
}, nil
}
callDescribeIndex := func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
callDescribeIndex := func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
IndexInfos: []*datapb.IndexInfo{
{},
},
}, nil
}
callGetSegmentIndexState := func(ctx context.Context, collID UniqueID, indexName string,
segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
return []*indexpb.SegmentIndexState{
segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
return []*datapb.SegmentIndexState{
{
SegmentID: 201,
State: commonpb.IndexState_Finished,
@ -544,8 +545,8 @@ func TestImportManager_TestFlipTaskStateLoop(t *testing.T) {
defer wg.Done()
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
callDescribeIndex = func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
callDescribeIndex = func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
@ -566,8 +567,8 @@ func TestImportManager_TestFlipTaskStateLoop(t *testing.T) {
defer wg.Done()
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
callDescribeIndex = func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
callDescribeIndex = func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_IndexNotExist,
},
@ -1233,10 +1234,10 @@ func TestImportManager_checkIndexingDone(t *testing.T) {
ctx := context.Background()
mgr := &importManager{
callDescribeIndex: func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
callDescribeIndex: func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return nil, errors.New("error")
},
callGetSegmentIndexState: func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
callGetSegmentIndexState: func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
return nil, errors.New("error")
},
callGetSegmentStates: func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
@ -1255,8 +1256,8 @@ func TestImportManager_checkIndexingDone(t *testing.T) {
assert.False(t, done)
assert.Error(t, err)
mgr.callDescribeIndex = func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
mgr.callDescribeIndex = func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
@ -1271,8 +1272,8 @@ func TestImportManager_checkIndexingDone(t *testing.T) {
assert.False(t, done)
assert.Error(t, err)
mgr.callDescribeIndex = func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
mgr.callDescribeIndex = func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_IndexNotExist,
},
@ -1308,12 +1309,12 @@ func TestImportManager_checkIndexingDone(t *testing.T) {
assert.True(t, done)
assert.Nil(t, err)
mgr.callDescribeIndex = func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
mgr.callDescribeIndex = func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
IndexInfos: []*datapb.IndexInfo{
{
State: commonpb.IndexState_Finished,
},
@ -1329,8 +1330,8 @@ func TestImportManager_checkIndexingDone(t *testing.T) {
assert.False(t, done)
assert.Error(t, err)
mgr.callGetSegmentIndexState = func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
return []*indexpb.SegmentIndexState{
mgr.callGetSegmentIndexState = func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
return []*datapb.SegmentIndexState{
{
State: commonpb.IndexState_Finished,
},
@ -1345,8 +1346,8 @@ func TestImportManager_checkIndexingDone(t *testing.T) {
assert.False(t, done)
assert.Nil(t, err)
mgr.callGetSegmentIndexState = func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
return []*indexpb.SegmentIndexState{
mgr.callGetSegmentIndexState = func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
return []*datapb.SegmentIndexState{
{
State: commonpb.IndexState_Finished,
},

View File

@ -14,7 +14,6 @@ import (
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/tso"
@ -131,28 +130,28 @@ func newMockMetaTable() *mockMetaTable {
return &mockMetaTable{}
}
type mockIndexCoord struct {
types.IndexCoord
GetComponentStatesFunc func(ctx context.Context) (*milvuspb.ComponentStates, error)
GetSegmentIndexStateFunc func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error)
DropIndexFunc func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error)
}
func newMockIndexCoord() *mockIndexCoord {
return &mockIndexCoord{}
}
func (m mockIndexCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return m.GetComponentStatesFunc(ctx)
}
func (m mockIndexCoord) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return m.GetSegmentIndexStateFunc(ctx, req)
}
func (m mockIndexCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return m.DropIndexFunc(ctx, req)
}
//type mockIndexCoord struct {
// types.IndexCoord
// GetComponentStatesFunc func(ctx context.Context) (*milvuspb.ComponentStates, error)
// GetSegmentIndexStateFunc func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error)
// DropIndexFunc func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error)
//}
//
//func newMockIndexCoord() *mockIndexCoord {
// return &mockIndexCoord{}
//}
//
//func (m mockIndexCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
// return m.GetComponentStatesFunc(ctx)
//}
//
//func (m mockIndexCoord) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
// return m.GetSegmentIndexStateFunc(ctx, req)
//}
//
//func (m mockIndexCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
// return m.DropIndexFunc(ctx, req)
//}
type mockDataCoord struct {
types.DataCoord
@ -164,6 +163,8 @@ type mockDataCoord struct {
ImportFunc func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
UnsetIsImportingStateFunc func(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
broadCastAlteredCollectionFunc func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error)
GetSegmentIndexStateFunc func(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error)
DropIndexFunc func(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error)
}
func newMockDataCoord() *mockDataCoord {
@ -208,6 +209,14 @@ func (m *mockDataCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHeal
}, nil
}
func (m *mockDataCoord) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return m.GetSegmentIndexStateFunc(ctx, req)
}
func (m *mockDataCoord) DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return m.DropIndexFunc(ctx, req)
}
type mockQueryCoord struct {
types.QueryCoord
GetSegmentInfoFunc func(ctx context.Context, req *querypb.GetSegmentInfoRequest) (*querypb.GetSegmentInfoResponse, error)
@ -479,74 +488,6 @@ func withValidQueryCoord() Opt {
return withQueryCoord(qc)
}
func withIndexCoord(ic types.IndexCoord) Opt {
return func(c *Core) {
c.indexCoord = ic
}
}
func withUnhealthyIndexCoord() Opt {
ic := newMockIndexCoord()
ic.GetComponentStatesFunc = func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Abnormal},
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "error mock GetComponentStates"),
}, retry.Unrecoverable(errors.New("error mock GetComponentStates"))
}
return withIndexCoord(ic)
}
func withInvalidIndexCoord() Opt {
ic := newMockIndexCoord()
ic.GetComponentStatesFunc = func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: succStatus(),
}, nil
}
ic.GetSegmentIndexStateFunc = func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return &indexpb.GetSegmentIndexStateResponse{}, errors.New("error mock GetSegmentIndexState")
}
ic.DropIndexFunc = func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return succStatus(), errors.New("error mock DropIndex")
}
return withIndexCoord(ic)
}
func withFailedIndexCoord() Opt {
ic := newMockIndexCoord()
ic.GetComponentStatesFunc = func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: succStatus(),
}, nil
}
ic.GetSegmentIndexStateFunc = func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return &indexpb.GetSegmentIndexStateResponse{Status: failStatus(commonpb.ErrorCode_UnexpectedError, "reason mock GetSegmentIndexState")}, nil
}
ic.DropIndexFunc = func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return failStatus(commonpb.ErrorCode_UnexpectedError, "reason mock DropIndex"), nil
}
return withIndexCoord(ic)
}
func withValidIndexCoord() Opt {
ic := newMockIndexCoord()
ic.GetComponentStatesFunc = func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: succStatus(),
}, nil
}
ic.GetSegmentIndexStateFunc = func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return &indexpb.GetSegmentIndexStateResponse{Status: succStatus()}, nil
}
ic.DropIndexFunc = func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return succStatus(), nil
}
return withIndexCoord(ic)
}
// cleanTestEnv clean test environment, for example, files generated by rocksmq.
func cleanTestEnv() {
path := "/tmp/milvus"
@ -627,6 +568,12 @@ func withInvalidDataCoord() Opt {
dc.broadCastAlteredCollectionFunc = func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
return nil, errors.New("error mock broadCastAlteredCollection")
}
dc.GetSegmentIndexStateFunc = func(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return nil, errors.New("error mock GetSegmentIndexStateFunc")
}
dc.DropIndexFunc = func(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return nil, errors.New("error mock DropIndexFunc")
}
return withDataCoord(dc)
}
@ -668,6 +615,14 @@ func withFailedDataCoord() Opt {
dc.broadCastAlteredCollectionFunc = func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
return failStatus(commonpb.ErrorCode_UnexpectedError, "mock broadcast altered collection error"), nil
}
dc.GetSegmentIndexStateFunc = func(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return &datapb.GetSegmentIndexStateResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError, "mock GetSegmentIndexStateFunc fail"),
}, nil
}
dc.DropIndexFunc = func(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return failStatus(commonpb.ErrorCode_UnexpectedError, "mock DropIndexFunc fail"), nil
}
return withDataCoord(dc)
}
@ -706,6 +661,14 @@ func withValidDataCoord() Opt {
dc.broadCastAlteredCollectionFunc = func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
return succStatus(), nil
}
dc.GetSegmentIndexStateFunc = func(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error) {
return &datapb.GetSegmentIndexStateResponse{
Status: succStatus(),
}, nil
}
dc.DropIndexFunc = func(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error) {
return succStatus(), nil
}
return withDataCoord(dc)
}
@ -824,8 +787,8 @@ type mockBroker struct {
ImportFunc func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
DropCollectionIndexFunc func(ctx context.Context, collID UniqueID, partIDs []UniqueID) error
DescribeIndexFunc func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error)
GetSegmentIndexStateFunc func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error)
DescribeIndexFunc func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error)
GetSegmentIndexStateFunc func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error)
BroadcastAlteredCollectionFunc func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error
}
@ -850,11 +813,11 @@ func (b mockBroker) DropCollectionIndex(ctx context.Context, collID UniqueID, pa
return b.DropCollectionIndexFunc(ctx, collID, partIDs)
}
func (b mockBroker) DescribeIndex(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
func (b mockBroker) DescribeIndex(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return b.DescribeIndexFunc(ctx, colID)
}
func (b mockBroker) GetSegmentIndexState(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
func (b mockBroker) GetSegmentIndexState(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*datapb.SegmentIndexState, error) {
return b.GetSegmentIndexStateFunc(ctx, collID, indexName, segIDs)
}

View File

@ -120,7 +120,6 @@ type Core struct {
dataCoord types.DataCoord
queryCoord types.QueryCoord
indexCoord types.IndexCoord
quotaCenter *QuotaCenter
@ -275,17 +274,6 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
return nil
}
func (c *Core) SetIndexCoord(s types.IndexCoord) error {
if err := s.Init(); err != nil {
return err
}
if err := s.Start(); err != nil {
return err
}
c.indexCoord = s
return nil
}
func (c *Core) SetQueryCoord(s types.QueryCoord) error {
if err := s.Init(); err != nil {
return err

View File

@ -19,7 +19,6 @@ import (
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
@ -1167,8 +1166,8 @@ func TestCore_ReportImport(t *testing.T) {
}, nil
}
callDescribeIndex := func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
callDescribeIndex := func(ctx context.Context, colID UniqueID) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_IndexNotExist,
},

View File

@ -19,12 +19,13 @@ package types
import (
"context"
"github.com/milvus-io/milvus/internal/proto/indexpb"
clientv3 "go.etcd.io/etcd/client/v3"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/querypb"
@ -318,9 +319,6 @@ type DataCoord interface {
// UpdateChannelCheckpoint updates channel checkpoint in dataCoord.
UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error)
AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error)
ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error)
// SaveImportSegment saves the import segment binlog paths data and then looks for the right DataNode to add the
// segment to that DataNode.
SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error)
@ -334,6 +332,34 @@ type DataCoord interface {
BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error)
CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error)
// CreateIndex create an index on collection.
// Index building is asynchronous, so when an index building request comes, an IndexID is assigned to the task and
// will get all flushed segments from DataCoord and record tasks with these segments. The background process
// indexBuilder will find this task and assign it to IndexNode for execution.
CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest) (*commonpb.Status, error)
// GetIndexState gets the index state of the index name in the request from Proxy.
// Deprecated: use DescribeIndex instead
GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest) (*datapb.GetIndexStateResponse, error)
// GetSegmentIndexState gets the index state of the segments in the request from RootCoord.
GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest) (*datapb.GetSegmentIndexStateResponse, error)
// GetIndexInfos gets the index files of the IndexBuildIDs in the request from RootCoordinator.
GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error)
// DescribeIndex describe the index info of the collection.
DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error)
// GetIndexBuildProgress get the index building progress by num rows.
// Deprecated: use DescribeIndex instead
GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest) (*datapb.GetIndexBuildProgressResponse, error)
// DropIndex deletes indexes based on IndexID. One IndexID corresponds to the index of an entire column. A column is
// divided into many segments, and each segment corresponds to an IndexBuildID. IndexCoord uses IndexBuildID to record
// index tasks. Therefore, when DropIndex is called, delete all tasks corresponding to IndexBuildID corresponding to IndexID.
DropIndex(ctx context.Context, req *datapb.DropIndexRequest) (*commonpb.Status, error)
}
// DataCoordComponent defines the interface of DataCoord component.
@ -344,8 +370,6 @@ type DataCoordComponent interface {
// SetEtcdClient set EtcdClient for DataCoord
// `etcdClient` is a client of etcd
SetEtcdClient(etcdClient *clientv3.Client)
SetIndexCoord(indexCoord IndexCoord)
}
// IndexNode is the interface `indexnode` package implements
@ -355,8 +379,8 @@ type IndexNode interface {
// BuildIndex receives request from IndexCoordinator to build an index.
// Index building is asynchronous, so when an index building request comes, IndexNode records the task and returns.
//BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequest) (*commonpb.Status, error)
//GetTaskSlots(ctx context.Context, req *indexpb.GetTaskSlotsRequest) (*indexpb.GetTaskSlotsResponse, error)
//BuildIndex(ctx context.Context, req *datapb.BuildIndexRequest) (*commonpb.Status, error)
//GetTaskSlots(ctx context.Context, req *datapb.GetTaskSlotsRequest) (*datapb.GetTaskSlotsResponse, error)
//
//// GetMetrics gets the metrics about IndexNode.
//GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
@ -808,12 +832,6 @@ type RootCoordComponent interface {
// Always return nil.
SetDataCoord(ctx context.Context, dataCoord DataCoord) error
// SetIndexCoord set IndexCoord for RootCoord
// `indexCoord` is a client of index coordinator.
//
// Always return nil.
SetIndexCoord(indexCoord IndexCoord) error
// SetQueryCoord set QueryCoord for RootCoord
// `queryCoord` is a client of query coordinator.
//
@ -886,7 +904,7 @@ type ProxyComponent interface {
// SetIndexCoordClient set IndexCoord for Proxy
// `indexCoord` is a client of index coordinator.
SetIndexCoordClient(indexCoord IndexCoord)
//SetIndexCoordClient(indexCoord IndexCoord)
// SetQueryCoordClient set QueryCoord for Proxy
// `queryCoord` is a client of query coordinator.

View File

@ -30,8 +30,12 @@ const (
SegmentMetaPrefix = "queryCoord-segmentMeta"
ChangeInfoMetaPrefix = "queryCoord-sealedSegmentChangeInfo"
// FlushedSegmentPrefix TODO @cai.zhang: remove this
FlushedSegmentPrefix = "flushed-segment"
// HandoffSegmentPrefix TODO @cai.zhang: remove this
HandoffSegmentPrefix = "querycoord-handoff"
// SegmentReferPrefix TODO @cai.zhang: remove this
SegmentReferPrefix = "segmentRefer"
SegmentIndexPrefix = "segment-index"
FieldIndexPrefix = "field-index"

View File

@ -173,3 +173,35 @@ func (m *GrpcDataCoordClient) BroadcastAlteredCollection(ctx context.Context, in
return &commonpb.Status{}, m.Err
}
func (m *GrpcDataCoordClient) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcDataCoordClient) DropIndex(ctx context.Context, req *datapb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcDataCoordClient) GetIndexState(ctx context.Context, req *datapb.GetIndexStateRequest, opts ...grpc.CallOption) (*datapb.GetIndexStateResponse, error) {
return &datapb.GetIndexStateResponse{}, m.Err
}
// GetSegmentIndexState gets the index state of the segments in the request from RootCoord.
func (m *GrpcDataCoordClient) GetSegmentIndexState(ctx context.Context, req *datapb.GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*datapb.GetSegmentIndexStateResponse, error) {
return &datapb.GetSegmentIndexStateResponse{}, m.Err
}
// GetIndexInfos gets the index files of the IndexBuildIDs in the request from RootCoordinator.
func (m *GrpcDataCoordClient) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest, opts ...grpc.CallOption) (*datapb.GetIndexInfoResponse, error) {
return &datapb.GetIndexInfoResponse{}, m.Err
}
// DescribeIndex describe the index info of the collection.
func (m *GrpcDataCoordClient) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest, opts ...grpc.CallOption) (*datapb.DescribeIndexResponse, error) {
return &datapb.DescribeIndexResponse{}, m.Err
}
// GetIndexBuildProgress get the index building progress by num rows.
func (m *GrpcDataCoordClient) GetIndexBuildProgress(ctx context.Context, req *datapb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*datapb.GetIndexBuildProgressResponse, error) {
return &datapb.GetIndexBuildProgressResponse{}, m.Err
}

View File

@ -1331,6 +1331,13 @@ type dataCoordConfig struct {
GCMissingTolerance ParamItem `refreshable:"false"`
GCDropTolerance ParamItem `refreshable:"false"`
EnableActiveStandby ParamItem `refreshable:"false"`
BindIndexNodeMode ParamItem `refreshable:"false"`
IndexNodeAddress ParamItem `refreshable:"false"`
WithCredential ParamItem `refreshable:"false"`
IndexNodeID ParamItem `refreshable:"false"`
MinSegmentNumRowsToEnableIndex ParamItem `refreshable:"true"`
}
func (p *dataCoordConfig) init(base *BaseTable) {
@ -1530,6 +1537,41 @@ func (p *dataCoordConfig) init(base *BaseTable) {
DefaultValue: "false",
}
p.EnableActiveStandby.Init(base.mgr)
p.MinSegmentNumRowsToEnableIndex = ParamItem{
Key: "dataCoord.segment.minSegmentNumRowsToEnableIndex",
Version: "2.0.0",
DefaultValue: "1024",
}
p.MinSegmentNumRowsToEnableIndex.Init(base.mgr)
p.BindIndexNodeMode = ParamItem{
Key: "dataCoord.bindIndexNodeMode.enable",
Version: "2.0.0",
DefaultValue: "false",
}
p.BindIndexNodeMode.Init(base.mgr)
p.IndexNodeAddress = ParamItem{
Key: "dataCoord.bindIndexNodeMode.address",
Version: "2.0.0",
DefaultValue: "localhost:22930",
}
p.IndexNodeAddress.Init(base.mgr)
p.WithCredential = ParamItem{
Key: "dataCoord.bindIndexNodeMode.withCred",
Version: "2.0.0",
DefaultValue: "false",
}
p.WithCredential.Init(base.mgr)
p.IndexNodeID = ParamItem{
Key: "dataCoord.bindIndexNodeMode.nodeID",
Version: "2.0.0",
DefaultValue: "0",
}
p.IndexNodeID.Init(base.mgr)
}
// /////////////////////////////////////////////////////////////////////////////

View File

@ -147,10 +147,10 @@ go test -race -cover ${APPLE_SILICON_FLAG} "${MILVUS_DIR}/distributed/querycoord
go test -race -cover ${APPLE_SILICON_FLAG} "${MILVUS_DIR}/querycoordv2/..." -failfast
}
function test_indexcoord()
{
go test -race -cover ${APPLE_SILICON_FLAG} "${MILVUS_DIR}/indexcoord/..." -failfast
}
#function test_indexcoord()
#{
#go test -race -cover ${APPLE_SILICON_FLAG} "${MILVUS_DIR}/indexcoord/..." -failfast
#}
function test_metastore()
{
@ -166,7 +166,7 @@ test_indexnode
test_rootcoord
test_querycoord
test_datacoord
test_indexcoord
#test_indexcoord
test_kv
test_mq
test_storage
@ -201,9 +201,9 @@ case "${TEST_TAG}" in
datacoord)
test_datacoord
;;
indexcoord)
test_indexcoord
;;
# indexcoord)
# test_indexcoord
# ;;
kv)
test_kv
;;