mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-02 03:48:37 +08:00
Merge IndexCoord and DataCoord (#21267)
Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
This commit is contained in:
parent
7b39873ae0
commit
e5f408dceb
@ -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")
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
28
internal/datacoord/build_index_policy.go
Normal file
28
internal/datacoord/build_index_policy.go
Normal 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]
|
||||
})
|
||||
|
||||
}
|
@ -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{}),
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
@ -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) ||
|
||||
|
@ -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,
|
||||
},
|
||||
|
@ -25,3 +25,10 @@ const (
|
||||
const (
|
||||
moduleName = "DataCoord"
|
||||
)
|
||||
|
||||
const (
|
||||
flatIndex = "FLAT"
|
||||
binFlatIndex = "BIN_FLAT"
|
||||
diskAnnIndex = "DISKANN"
|
||||
invalidIndex = "invalid"
|
||||
)
|
||||
|
@ -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"
|
||||
}
|
||||
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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(),
|
||||
}
|
||||
}
|
||||
|
||||
|
449
internal/datacoord/index_builder.go
Normal file
449
internal/datacoord/index_builder.go
Normal 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
|
||||
}
|
||||
}
|
||||
}
|
1015
internal/datacoord/index_builder_test.go
Normal file
1015
internal/datacoord/index_builder_test.go
Normal file
File diff suppressed because it is too large
Load Diff
689
internal/datacoord/index_meta.go
Normal file
689
internal/datacoord/index_meta.go
Normal 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
|
||||
}
|
1178
internal/datacoord/index_meta_test.go
Normal file
1178
internal/datacoord/index_meta_test.go
Normal file
File diff suppressed because it is too large
Load Diff
599
internal/datacoord/index_service.go
Normal file
599
internal/datacoord/index_service.go
Normal 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
|
||||
}
|
1085
internal/datacoord/index_service_test.go
Normal file
1085
internal/datacoord/index_service_test.go
Normal file
File diff suppressed because it is too large
Load Diff
255
internal/datacoord/indexnode_manager.go
Normal file
255
internal/datacoord/indexnode_manager.go
Normal 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
|
||||
}
|
270
internal/datacoord/indexnode_manager_test.go
Normal file
270
internal/datacoord/indexnode_manager_test.go
Normal 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))
|
||||
}
|
@ -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
|
||||
}
|
||||
|
@ -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{
|
||||
{
|
||||
|
@ -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),
|
||||
},
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
}
|
@ -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)
|
||||
})
|
||||
}
|
@ -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
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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",
|
||||
|
@ -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) {
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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) {
|
||||
|
@ -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")
|
||||
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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)
|
||||
|
@ -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))
|
||||
|
@ -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{}
|
||||
}
|
||||
|
@ -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
@ -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
|
||||
|
@ -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()
|
||||
//}
|
||||
|
@ -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
|
||||
|
@ -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 (
|
||||
|
@ -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 {
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
@ -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 {
|
||||
|
@ -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()),
|
||||
|
@ -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{
|
||||
|
@ -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)
|
||||
|
@ -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{}
|
||||
}
|
@ -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()
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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: "",
|
||||
})
|
||||
|
@ -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,
|
||||
})
|
||||
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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},
|
||||
})
|
||||
|
@ -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,
|
||||
})
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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},
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
@ -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",
|
||||
|
@ -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,
|
||||
},
|
||||
|
@ -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)
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
},
|
||||
|
@ -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.
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
||||
// /////////////////////////////////////////////////////////////////////////////
|
||||
|
@ -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
|
||||
;;
|
||||
|
Loading…
Reference in New Issue
Block a user