mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 01:28:27 +08:00
Add distributed lock for segment refgerence (#16782)
Signed-off-by: Cai.Zhang <cai.zhang@zilliz.com>
This commit is contained in:
parent
affcc2acad
commit
bcf3b7426a
27
internal/datacoord/const.go
Normal file
27
internal/datacoord/const.go
Normal file
@ -0,0 +1,27 @@
|
||||
// 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
|
||||
|
||||
// segment reference lock
|
||||
const (
|
||||
// segmentReferPrefix is the prefix of the segment reference lock path
|
||||
segmentReferPrefix = "segmentRefer"
|
||||
)
|
||||
|
||||
const (
|
||||
moduleName = "DataCoord"
|
||||
)
|
||||
@ -50,8 +50,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
|
||||
option GcOption
|
||||
meta *meta
|
||||
segRefer *SegmentReferenceManager
|
||||
|
||||
startOnce sync.Once
|
||||
stopOnce sync.Once
|
||||
@ -60,13 +61,14 @@ type garbageCollector struct {
|
||||
}
|
||||
|
||||
// newGarbageCollector create garbage collector with meta and option
|
||||
func newGarbageCollector(meta *meta, opt GcOption) *garbageCollector {
|
||||
func newGarbageCollector(meta *meta, segRefer *SegmentReferenceManager, 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,
|
||||
option: opt,
|
||||
closeCh: make(chan struct{}),
|
||||
meta: meta,
|
||||
segRefer: segRefer,
|
||||
option: opt,
|
||||
closeCh: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
@ -134,6 +136,15 @@ func (gc *garbageCollector) scan() {
|
||||
v++
|
||||
continue
|
||||
}
|
||||
|
||||
// binlog path should consist of "/files/insertLog/collID/partID/segID/fieldID/fileName"
|
||||
segmentID, err := parseSegmentIDByBinlog(info.Key)
|
||||
if err == nil {
|
||||
if gc.segRefer.HasSegmentLock(segmentID) {
|
||||
v++
|
||||
continue
|
||||
}
|
||||
}
|
||||
m++
|
||||
// not found in meta, check last modified time exceeds tolerance duration
|
||||
if time.Since(info.LastModified) > gc.option.missingTolerance {
|
||||
@ -148,7 +159,7 @@ func (gc *garbageCollector) scan() {
|
||||
|
||||
func (gc *garbageCollector) clearEtcd() {
|
||||
drops := gc.meta.SelectSegments(func(segment *SegmentInfo) bool {
|
||||
return segment.GetState() == commonpb.SegmentState_Dropped
|
||||
return segment.GetState() == commonpb.SegmentState_Dropped && !gc.segRefer.HasSegmentLock(segment.ID)
|
||||
})
|
||||
|
||||
for _, sinfo := range drops {
|
||||
|
||||
@ -20,12 +20,15 @@ import (
|
||||
"bytes"
|
||||
"context"
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"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"
|
||||
@ -44,8 +47,15 @@ func Test_garbageCollector_basic(t *testing.T) {
|
||||
meta, err := newMemoryMeta(mockAllocator)
|
||||
assert.Nil(t, err)
|
||||
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
segRefer, err := NewSegmentReferenceManager(etcdKV, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, segRefer)
|
||||
|
||||
t.Run("normal gc", func(t *testing.T) {
|
||||
gc := newGarbageCollector(meta, GcOption{
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: cli,
|
||||
enabled: true,
|
||||
checkInterval: time.Millisecond * 10,
|
||||
@ -63,7 +73,7 @@ func Test_garbageCollector_basic(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("with nil cli", func(t *testing.T) {
|
||||
gc := newGarbageCollector(meta, GcOption{
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: nil,
|
||||
enabled: true,
|
||||
checkInterval: time.Millisecond * 10,
|
||||
@ -102,8 +112,50 @@ func Test_garbageCollector_scan(t *testing.T) {
|
||||
meta, err := newMemoryMeta(mockAllocator)
|
||||
assert.Nil(t, err)
|
||||
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
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][]*SegmentLock{
|
||||
2: {
|
||||
{
|
||||
segmentID: 2,
|
||||
nodeID: 1,
|
||||
locKey: "path",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: cli,
|
||||
enabled: true,
|
||||
checkInterval: time.Minute * 30,
|
||||
missingTolerance: time.Hour * 24,
|
||||
dropTolerance: time.Hour * 24,
|
||||
bucketName: bucketName,
|
||||
rootPath: rootPath,
|
||||
})
|
||||
gc.segRefer = segReferManager
|
||||
gc.scan()
|
||||
|
||||
validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, insertLogPrefix), inserts)
|
||||
validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, statsLogPrefix), stats)
|
||||
validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, deltaLogPrefix), delta)
|
||||
validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, `indexes`), others)
|
||||
|
||||
err = gc.segRefer.ReleaseSegmentsLock([]UniqueID{2}, 1)
|
||||
assert.NoError(t, err)
|
||||
gc.close()
|
||||
})
|
||||
|
||||
t.Run("missing all but save tolerance", func(t *testing.T) {
|
||||
gc := newGarbageCollector(meta, GcOption{
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: cli,
|
||||
enabled: true,
|
||||
checkInterval: time.Minute * 30,
|
||||
@ -130,7 +182,7 @@ func Test_garbageCollector_scan(t *testing.T) {
|
||||
err = meta.AddSegment(segment)
|
||||
require.NoError(t, err)
|
||||
|
||||
gc := newGarbageCollector(meta, GcOption{
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: cli,
|
||||
enabled: true,
|
||||
checkInterval: time.Minute * 30,
|
||||
@ -160,7 +212,7 @@ func Test_garbageCollector_scan(t *testing.T) {
|
||||
err = meta.AddSegment(segment)
|
||||
require.NoError(t, err)
|
||||
|
||||
gc := newGarbageCollector(meta, GcOption{
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: cli,
|
||||
enabled: true,
|
||||
checkInterval: time.Minute * 30,
|
||||
@ -178,7 +230,7 @@ func Test_garbageCollector_scan(t *testing.T) {
|
||||
gc.close()
|
||||
})
|
||||
t.Run("missing gc all", func(t *testing.T) {
|
||||
gc := newGarbageCollector(meta, GcOption{
|
||||
gc := newGarbageCollector(meta, segRefer, GcOption{
|
||||
cli: cli,
|
||||
enabled: true,
|
||||
checkInterval: time.Minute * 30,
|
||||
@ -229,7 +281,10 @@ func initUtOSSEnv(bucket, root string, n int) (cli *minio.Client, inserts []stri
|
||||
content := []byte("test")
|
||||
for i := 0; i < n; i++ {
|
||||
reader := bytes.NewReader(content)
|
||||
token := funcutil.RandomString(8)
|
||||
token := path.Join(funcutil.RandomString(8), funcutil.RandomString(8), strconv.Itoa(i), funcutil.RandomString(8), funcutil.RandomString(8))
|
||||
if i == 1 {
|
||||
token = path.Join(funcutil.RandomString(8), funcutil.RandomString(8), strconv.Itoa(i), funcutil.RandomString(8))
|
||||
}
|
||||
// insert
|
||||
filePath := path.Join(root, insertLogPrefix, token)
|
||||
info, err := cli.PutObject(context.TODO(), bucket, filePath, reader, int64(len(content)), minio.PutObjectOptions{})
|
||||
|
||||
@ -1052,3 +1052,15 @@ func isSegmentHealthy(segment *SegmentInfo) bool {
|
||||
segment.GetState() != commonpb.SegmentState_NotExist &&
|
||||
segment.GetState() != commonpb.SegmentState_Dropped
|
||||
}
|
||||
|
||||
func (m *meta) HasSegments(segIDs []UniqueID) (bool, error) {
|
||||
m.RLock()
|
||||
defer m.RUnlock()
|
||||
|
||||
for _, segID := range segIDs {
|
||||
if _, ok := m.segments.segments[segID]; !ok {
|
||||
return false, fmt.Errorf("segment is not exist with ID = %d", segID)
|
||||
}
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
@ -698,3 +698,26 @@ func Test_meta_GetSegmentsOfCollection(t *testing.T) {
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestMeta_HasSegments(t *testing.T) {
|
||||
m := &meta{
|
||||
segments: &SegmentsInfo{
|
||||
segments: map[UniqueID]*SegmentInfo{
|
||||
1: {
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 1,
|
||||
},
|
||||
currRows: 100,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
has, err := m.HasSegments([]UniqueID{1})
|
||||
assert.Equal(t, true, has)
|
||||
assert.NoError(t, err)
|
||||
|
||||
has, err = m.HasSegments([]UniqueID{2})
|
||||
assert.Equal(t, false, has)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
244
internal/datacoord/segment_reference_manager.go
Normal file
244
internal/datacoord/segment_reference_manager.go
Normal file
@ -0,0 +1,244 @@
|
||||
// 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 (
|
||||
"fmt"
|
||||
"path"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
)
|
||||
|
||||
type SegmentLock struct {
|
||||
segmentID UniqueID
|
||||
nodeID UniqueID
|
||||
locKey string
|
||||
}
|
||||
|
||||
type SegmentReferenceManager struct {
|
||||
etcdKV kv.BaseKV
|
||||
|
||||
segmentsLock map[UniqueID][]*SegmentLock
|
||||
lock sync.RWMutex
|
||||
}
|
||||
|
||||
func parseLockKey(key string) (segID UniqueID, nodeID UniqueID, err error) {
|
||||
ss := strings.Split(key, "/")
|
||||
// segment lock key consists of at least "meta/segmentRefer/nodeID/segID"
|
||||
if len(ss) < 4 {
|
||||
return 0, 0, fmt.Errorf("segment lock key is invalid with %s", key)
|
||||
}
|
||||
segID, err = strconv.ParseInt(ss[len(ss)-1], 10, 64)
|
||||
if err != nil {
|
||||
return 0, 0, err
|
||||
}
|
||||
nodeID, err = strconv.ParseInt(ss[len(ss)-2], 10, 64)
|
||||
return segID, nodeID, err
|
||||
}
|
||||
|
||||
func NewSegmentReferenceManager(etcdKV kv.BaseKV, onlineIDs []UniqueID) (*SegmentReferenceManager, error) {
|
||||
log.Info("New segment reference manager")
|
||||
segReferManager := &SegmentReferenceManager{
|
||||
etcdKV: etcdKV,
|
||||
segmentsLock: make(map[UniqueID][]*SegmentLock),
|
||||
}
|
||||
keys, _, err := segReferManager.etcdKV.LoadWithPrefix(segmentReferPrefix)
|
||||
if err != nil {
|
||||
log.Error("load segments lock from etcd failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, key := range keys {
|
||||
segID, nodeID, err := parseLockKey(key)
|
||||
if err != nil {
|
||||
log.Error("parse segment lock key failed", zap.String("lock key", key), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
segLock := &SegmentLock{
|
||||
segmentID: segID,
|
||||
nodeID: nodeID,
|
||||
locKey: key,
|
||||
}
|
||||
segReferManager.segmentsLock[segID] = append(segReferManager.segmentsLock[segID], segLock)
|
||||
}
|
||||
|
||||
err = segReferManager.recoverySegReferManager(onlineIDs)
|
||||
if err != nil {
|
||||
log.Error("Recovery segment reference manager failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
log.Info("New segment reference manager successfully")
|
||||
return segReferManager, nil
|
||||
}
|
||||
|
||||
func (srm *SegmentReferenceManager) AddSegmentsLock(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))
|
||||
locKVs := make(map[string]string)
|
||||
segID2SegmentLock := make(map[UniqueID][]*SegmentLock)
|
||||
for _, segID := range segIDs {
|
||||
locKey := path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), strconv.FormatInt(segID, 10))
|
||||
locKVs[locKey] = strconv.FormatInt(nodeID, 10)
|
||||
segLock := &SegmentLock{
|
||||
segmentID: segID,
|
||||
nodeID: nodeID,
|
||||
locKey: locKey,
|
||||
}
|
||||
segID2SegmentLock[segID] = append(segID2SegmentLock[segID], segLock)
|
||||
}
|
||||
if err := srm.etcdKV.MultiSave(locKVs); err != nil {
|
||||
log.Error("AddSegmentsLock save segment lock to etcd failed", zap.Int64s("segIDs", segIDs), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
for segID, segLocks := range segID2SegmentLock {
|
||||
srm.segmentsLock[segID] = append(srm.segmentsLock[segID], segLocks...)
|
||||
}
|
||||
log.Info("Add reference lock on segments successfully", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (srm *SegmentReferenceManager) ReleaseSegmentsLock(segIDs []UniqueID, nodeID UniqueID) error {
|
||||
srm.lock.Lock()
|
||||
defer srm.lock.Unlock()
|
||||
|
||||
log.Info("Release reference lock on segments", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID))
|
||||
locKeys := make([]string, 0)
|
||||
for _, segID := range segIDs {
|
||||
if _, ok := srm.segmentsLock[segID]; !ok {
|
||||
continue
|
||||
}
|
||||
for _, segLock := range srm.segmentsLock[segID] {
|
||||
if segLock.nodeID == nodeID {
|
||||
locKeys = append(locKeys, segLock.locKey)
|
||||
}
|
||||
}
|
||||
}
|
||||
if err := srm.etcdKV.MultiRemove(locKeys); err != nil {
|
||||
log.Error("Remove reference lock paths on segments failed", zap.Int64s("segIDs", segIDs),
|
||||
zap.Int64("nodeID", nodeID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
for _, segID := range segIDs {
|
||||
if _, ok := srm.segmentsLock[segID]; !ok {
|
||||
continue
|
||||
}
|
||||
for i := 0; i < len(srm.segmentsLock[segID]); i++ {
|
||||
segLock := srm.segmentsLock[segID][i]
|
||||
if segLock.nodeID == nodeID {
|
||||
srm.segmentsLock[segID] = append(srm.segmentsLock[segID][:i], srm.segmentsLock[segID][i+1:]...)
|
||||
i--
|
||||
}
|
||||
}
|
||||
if len(srm.segmentsLock[segID]) == 0 {
|
||||
delete(srm.segmentsLock, segID)
|
||||
}
|
||||
}
|
||||
log.Info("Release reference lock on segments successfully", zap.Int64s("segIDs", segIDs), 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))
|
||||
locKeys := make([]string, 0)
|
||||
for segID := range srm.segmentsLock {
|
||||
for _, segLock := range srm.segmentsLock[segID] {
|
||||
if segLock.nodeID == nodeID {
|
||||
locKeys = append(locKeys, segLock.locKey)
|
||||
}
|
||||
}
|
||||
}
|
||||
if err := srm.etcdKV.MultiRemove(locKeys); err != nil {
|
||||
log.Error("Remove reference lock paths on segments by node failed",
|
||||
zap.Int64("nodeID", nodeID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
for segID := range srm.segmentsLock {
|
||||
for i := 0; i < len(srm.segmentsLock[segID]); i++ {
|
||||
segLock := srm.segmentsLock[segID][i]
|
||||
if segLock.nodeID == nodeID {
|
||||
srm.segmentsLock[segID] = append(srm.segmentsLock[segID][:i], srm.segmentsLock[segID][i+1:]...)
|
||||
i--
|
||||
}
|
||||
}
|
||||
if len(srm.segmentsLock[segID]) == 0 {
|
||||
delete(srm.segmentsLock, segID)
|
||||
}
|
||||
}
|
||||
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))
|
||||
offlineIDs := make(map[UniqueID]struct{})
|
||||
for segID := range srm.segmentsLock {
|
||||
for _, segLock := range srm.segmentsLock[segID] {
|
||||
alive := false
|
||||
for _, nodeID := range nodeIDs {
|
||||
if segLock.nodeID == nodeID {
|
||||
alive = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !alive {
|
||||
offlineIDs[segLock.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()
|
||||
|
||||
_, ok := srm.segmentsLock[segID]
|
||||
return ok
|
||||
}
|
||||
|
||||
func (srm *SegmentReferenceManager) GetHasReferLockSegmentIDs() []UniqueID {
|
||||
srm.lock.RLock()
|
||||
defer srm.lock.RUnlock()
|
||||
|
||||
segIDs := make([]UniqueID, 0)
|
||||
for segID := range srm.segmentsLock {
|
||||
segIDs = append(segIDs, segID)
|
||||
}
|
||||
return segIDs
|
||||
}
|
||||
299
internal/datacoord/segment_reference_manager_test.go
Normal file
299
internal/datacoord/segment_reference_manager_test.go
Normal file
@ -0,0 +1,299 @@
|
||||
// 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/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)
|
||||
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)
|
||||
locKey = path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), strconv.FormatInt(2, 10))
|
||||
err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10))
|
||||
assert.NoError(t, err)
|
||||
|
||||
segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID})
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, segRefer)
|
||||
err = etcdKV.Remove(locKey)
|
||||
assert.NoError(t, err)
|
||||
|
||||
locKey = path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), "segID")
|
||||
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)
|
||||
|
||||
locKey = path.Join(segmentReferPrefix, "nodeID", strconv.FormatInt(3, 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)
|
||||
|
||||
locKey = path.Join(segmentReferPrefix, "nodeID")
|
||||
err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10))
|
||||
assert.NoError(t, err)
|
||||
segRefer, err = NewSegmentReferenceManager(etcdKV, nil)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, segRefer)
|
||||
err = etcdKV.Remove(locKey)
|
||||
assert.NoError(t, err)
|
||||
|
||||
locKey = path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), strconv.FormatInt(2, 10))
|
||||
err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10))
|
||||
assert.NoError(t, err)
|
||||
segRefer, err = NewSegmentReferenceManager(etcdKV, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, segRefer)
|
||||
has := segRefer.HasSegmentLock(2)
|
||||
assert.False(t, has)
|
||||
})
|
||||
|
||||
segIDs := []UniqueID{1, 2, 3, 4, 5}
|
||||
nodeID := UniqueID(1)
|
||||
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(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(segIDs, nodeID)
|
||||
assert.NoError(t, err)
|
||||
|
||||
for _, segID := range segIDs {
|
||||
has = segRefer.HasSegmentLock(segID)
|
||||
assert.False(t, has)
|
||||
}
|
||||
|
||||
err = segRefer.ReleaseSegmentsLock([]UniqueID{6}, 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
|
||||
err = segRefer.AddSegmentsLock(segIDs, nodeID)
|
||||
assert.NoError(t, err)
|
||||
|
||||
for _, segID := range segIDs {
|
||||
has = segRefer.HasSegmentLock(segID)
|
||||
assert.True(t, has)
|
||||
}
|
||||
|
||||
err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(2))
|
||||
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}
|
||||
err = segRefer.AddSegmentsLock(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(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}
|
||||
err = segRefer.AddSegmentsLock(segIDs, UniqueID(5))
|
||||
assert.NoError(t, err)
|
||||
|
||||
for _, segID := range segIDs {
|
||||
has = segRefer.HasSegmentLock(segID)
|
||||
assert.True(t, has)
|
||||
}
|
||||
|
||||
segmentIDs := segRefer.GetHasReferLockSegmentIDs()
|
||||
assert.Equal(t, 5, len(segmentIDs))
|
||||
assert.ElementsMatch(t, segIDs, segmentIDs)
|
||||
|
||||
err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(5))
|
||||
assert.NoError(t, err)
|
||||
|
||||
for _, segID := range segIDs {
|
||||
has = segRefer.HasSegmentLock(segID)
|
||||
assert.False(t, has)
|
||||
}
|
||||
|
||||
segIDs = segRefer.GetHasReferLockSegmentIDs()
|
||||
assert.Equal(t, 0, len(segIDs))
|
||||
})
|
||||
}
|
||||
|
||||
type etcdKVMock struct {
|
||||
kv.BaseKV
|
||||
|
||||
Fail int
|
||||
}
|
||||
|
||||
func (em *etcdKVMock) MultiSave(data map[string]string) error {
|
||||
if em.Fail > 0 {
|
||||
return errors.New("error occurred")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (em *etcdKVMock) MultiRemove(keys []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
|
||||
}
|
||||
return []string{"meta/segmentRefer/1/2"}, []string{"1"}, 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,
|
||||
}
|
||||
|
||||
t.Run("AddSegmentsLock", func(t *testing.T) {
|
||||
err := segRefer.AddSegmentsLock([]UniqueID{1}, 1)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("ReleaseSegmentsLock", func(t *testing.T) {
|
||||
err := segRefer.ReleaseSegmentsLock([]UniqueID{1}, 1)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("ReleaseSegmentsLockByNodeID", func(t *testing.T) {
|
||||
err := segRefer.ReleaseSegmentsLockByNodeID(1)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("recoverySegReferManager", func(t *testing.T) {
|
||||
segRefer.segmentsLock = map[UniqueID][]*SegmentLock{
|
||||
2: {
|
||||
{
|
||||
segmentID: 2,
|
||||
nodeID: 2,
|
||||
locKey: "1/2/3",
|
||||
},
|
||||
},
|
||||
}
|
||||
err := segRefer.recoverySegReferManager([]UniqueID{1})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
||||
@ -127,11 +127,15 @@ type Server struct {
|
||||
flushCh chan UniqueID
|
||||
factory dependency.Factory
|
||||
|
||||
session *sessionutil.Session
|
||||
eventCh <-chan *sessionutil.SessionEvent
|
||||
session *sessionutil.Session
|
||||
dnEventCh <-chan *sessionutil.SessionEvent
|
||||
icEventCh <-chan *sessionutil.SessionEvent
|
||||
qcEventCh <-chan *sessionutil.SessionEvent
|
||||
|
||||
dataNodeCreator dataNodeCreatorFunc
|
||||
rootCoordClientCreator rootCoordCreatorFunc
|
||||
|
||||
segReferManager *SegmentReferenceManager
|
||||
}
|
||||
|
||||
// ServerHelper datacoord server injection helper
|
||||
@ -380,7 +384,7 @@ func (s *Server) initGarbageCollection() error {
|
||||
}
|
||||
}
|
||||
|
||||
s.garbageCollector = newGarbageCollector(s.meta, GcOption{
|
||||
s.garbageCollector = newGarbageCollector(s.meta, s.segReferManager, GcOption{
|
||||
cli: cli,
|
||||
enabled: Params.DataCoordCfg.EnableGarbageCollection,
|
||||
bucketName: Params.MinioCfg.BucketName,
|
||||
@ -413,8 +417,31 @@ func (s *Server) initServiceDiscovery() error {
|
||||
s.cluster.Startup(s.ctx, datanodes)
|
||||
|
||||
// TODO implement rewatch logic
|
||||
s.eventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1, nil)
|
||||
return nil
|
||||
s.dnEventCh = s.session.WatchServices(typeutil.DataNodeRole, 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)
|
||||
if err != nil {
|
||||
log.Error("DataCoord get QueryCoord session failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
for _, session := range qcSessions {
|
||||
serverIDs = append(serverIDs, session.ServerID)
|
||||
}
|
||||
s.qcEventCh = s.session.WatchServices(typeutil.QueryCoordRole, qcRevision+1, nil)
|
||||
|
||||
s.segReferManager, err = NewSegmentReferenceManager(s.kvClient, serverIDs)
|
||||
return err
|
||||
}
|
||||
|
||||
func (s *Server) startSegmentManager() {
|
||||
@ -628,7 +655,7 @@ func (s *Server) watchService(ctx context.Context) {
|
||||
case <-ctx.Done():
|
||||
log.Info("watch service shutdown")
|
||||
return
|
||||
case event, ok := <-s.eventCh:
|
||||
case event, ok := <-s.dnEventCh:
|
||||
if !ok {
|
||||
// ErrCompacted in handled inside SessionWatcher
|
||||
// So there is some other error occurred, closing DataCoord server
|
||||
@ -649,6 +676,56 @@ func (s *Server) watchService(ctx context.Context) {
|
||||
}()
|
||||
return
|
||||
}
|
||||
case event, ok := <-s.icEventCh:
|
||||
if !ok {
|
||||
// ErrCompacted in handled inside SessionWatcher
|
||||
// So there is some other error occurred, closing DataCoord server
|
||||
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID))
|
||||
go s.Stop()
|
||||
if s.session.TriggerKill {
|
||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||
p.Signal(syscall.SIGINT)
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
switch event.EventType {
|
||||
case sessionutil.SessionAddEvent:
|
||||
log.Info("there is a new IndexCoord online", zap.Int64("serverID", event.Session.ServerID))
|
||||
|
||||
case sessionutil.SessionDelEvent:
|
||||
log.Warn("there is IndexCoord offline", zap.Int64("serverID", event.Session.ServerID))
|
||||
if err := retry.Do(ctx, func() error {
|
||||
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
|
||||
}, retry.Attempts(100)); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
case event, ok := <-s.qcEventCh:
|
||||
if !ok {
|
||||
// ErrCompacted in handled inside SessionWatcher
|
||||
// So there is some other error occurred, closing DataCoord server
|
||||
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID))
|
||||
go s.Stop()
|
||||
if s.session.TriggerKill {
|
||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||
p.Signal(syscall.SIGINT)
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
switch event.EventType {
|
||||
case sessionutil.SessionAddEvent:
|
||||
log.Info("there is a new QueryCoord online", zap.Int64("serverID", event.Session.ServerID))
|
||||
|
||||
case sessionutil.SessionDelEvent:
|
||||
log.Warn("there is QueryCoord offline", zap.Int64("serverID", event.Session.ServerID))
|
||||
if err := retry.Do(ctx, func() error {
|
||||
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
|
||||
}, retry.Attempts(100)); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -30,6 +30,8 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
@ -38,6 +40,7 @@ import (
|
||||
|
||||
"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/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
@ -627,7 +630,7 @@ func TestService_WatchServices(t *testing.T) {
|
||||
svr.serverLoopWg.Add(1)
|
||||
|
||||
ech := make(chan *sessionutil.SessionEvent)
|
||||
svr.eventCh = ech
|
||||
svr.dnEventCh = ech
|
||||
|
||||
flag := false
|
||||
closed := false
|
||||
@ -654,7 +657,7 @@ func TestService_WatchServices(t *testing.T) {
|
||||
ech = make(chan *sessionutil.SessionEvent)
|
||||
|
||||
flag = false
|
||||
svr.eventCh = ech
|
||||
svr.dnEventCh = ech
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
svr.serverLoopWg.Add(1)
|
||||
|
||||
@ -670,6 +673,128 @@ func TestService_WatchServices(t *testing.T) {
|
||||
assert.True(t, flag)
|
||||
}
|
||||
|
||||
func TestServer_watchCoord(t *testing.T) {
|
||||
Params.Init()
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
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{}{}
|
||||
}()
|
||||
|
||||
icCh <- &sessionutil.SessionEvent{
|
||||
EventType: sessionutil.SessionAddEvent,
|
||||
Session: &sessionutil.Session{
|
||||
ServerID: 1,
|
||||
},
|
||||
}
|
||||
icCh <- &sessionutil.SessionEvent{
|
||||
EventType: sessionutil.SessionDelEvent,
|
||||
Session: &sessionutil.Session{
|
||||
ServerID: 1,
|
||||
},
|
||||
}
|
||||
close(icCh)
|
||||
<-sigQuit
|
||||
svr.serverLoopWg.Wait()
|
||||
assert.True(t, closed)
|
||||
}
|
||||
|
||||
func TestServer_watchQueryCoord(t *testing.T) {
|
||||
Params.Init()
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
|
||||
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_GetMetrics(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
defer closeTestServer(t, svr)
|
||||
@ -2494,6 +2619,30 @@ func TestImport(t *testing.T) {
|
||||
assert.NoError(t, err)
|
||||
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{
|
||||
SegmentIDs: []UniqueID{1, 2},
|
||||
NodeID: UniqueID(1),
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
|
||||
})
|
||||
}
|
||||
|
||||
// https://github.com/milvus-io/milvus/issues/15659
|
||||
@ -2580,3 +2729,91 @@ func closeTestServer(t *testing.T, svr *Server) {
|
||||
err = svr.CleanMeta()
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func newTestServer2(t *testing.T, receiveCh chan interface{}, opts ...Option) *Server {
|
||||
var err error
|
||||
Params.Init()
|
||||
Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int())
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.Nil(t, err)
|
||||
sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot)
|
||||
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
|
||||
assert.Nil(t, err)
|
||||
|
||||
icSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli)
|
||||
icSession.Init(typeutil.IndexCoordRole, "localhost:31000", true, true)
|
||||
icSession.Register()
|
||||
|
||||
qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli)
|
||||
qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true)
|
||||
qcSession.Register()
|
||||
|
||||
svr := CreateServer(context.TODO(), factory, opts...)
|
||||
svr.SetEtcdClient(etcdCli)
|
||||
svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) {
|
||||
return newMockDataNodeClient(0, receiveCh)
|
||||
}
|
||||
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
|
||||
return newMockRootCoordService(), nil
|
||||
}
|
||||
|
||||
err = svr.Init()
|
||||
assert.Nil(t, err)
|
||||
err = svr.Start()
|
||||
assert.Nil(t, err)
|
||||
|
||||
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
|
||||
assert.Nil(t, err)
|
||||
|
||||
err = svr.Register()
|
||||
assert.Nil(t, err)
|
||||
|
||||
// Stop channal watch state watcher in tests
|
||||
if svr.channelManager != nil && svr.channelManager.stopChecker != nil {
|
||||
svr.channelManager.stopChecker()
|
||||
}
|
||||
|
||||
return svr
|
||||
}
|
||||
|
||||
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, 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, 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)
|
||||
}
|
||||
|
||||
@ -34,11 +34,10 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
"github.com/milvus-io/milvus/internal/util/trace"
|
||||
)
|
||||
|
||||
const moduleName = "DataCoord"
|
||||
|
||||
// checks whether server in Healthy State
|
||||
func (s *Server) isClosed() bool {
|
||||
return atomic.LoadInt64(&s.isServing) != ServerStateHealthy
|
||||
@ -666,13 +665,15 @@ func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS
|
||||
}
|
||||
ret := make([]UniqueID, 0, len(segmentIDs))
|
||||
for _, id := range segmentIDs {
|
||||
s := s.meta.GetSegment(id)
|
||||
if s != nil && s.GetState() != commonpb.SegmentState_Flushed {
|
||||
segment := s.meta.GetSegment(id)
|
||||
if segment != nil && segment.GetState() != commonpb.SegmentState_Flushed {
|
||||
continue
|
||||
}
|
||||
|
||||
// if this segment == nil, we assume this segment has been compacted and flushed
|
||||
ret = append(ret, id)
|
||||
}
|
||||
|
||||
resp.Segments = ret
|
||||
resp.Status.ErrorCode = commonpb.ErrorCode_Success
|
||||
return resp, nil
|
||||
@ -1073,3 +1074,66 @@ 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(Params.DataCoordCfg.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.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.SegmentIDs, 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(Params.DataCoordCfg.GetNodeID())
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
err := s.segReferManager.ReleaseSegmentsLock(req.SegmentIDs, req.NodeID)
|
||||
if err != nil {
|
||||
log.Error("DataCoord ReleaseSegmentLock failed", zap.Int64s("segmentIDs", req.SegmentIDs), zap.Int64("nodeID", req.NodeID),
|
||||
zap.Error(err))
|
||||
resp.Reason = err.Error()
|
||||
return resp, nil
|
||||
}
|
||||
resp.ErrorCode = commonpb.ErrorCode_Success
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
@ -19,6 +19,8 @@ package datacoord
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
@ -77,3 +79,9 @@ func getTimetravelReverseTime(ctx context.Context, allocator allocator) (*timetr
|
||||
tt := tsoutil.ComposeTS(ttpts.UnixNano()/int64(time.Millisecond), 0)
|
||||
return &timetravel{tt}, nil
|
||||
}
|
||||
|
||||
func parseSegmentIDByBinlog(path string) (UniqueID, error) {
|
||||
// binlog path should consist of "files/insertLog/collID/partID/segID/fieldID/fileName"
|
||||
keyStr := strings.Split(path, "/")
|
||||
return strconv.ParseInt(keyStr[len(keyStr)-3], 10, 64)
|
||||
}
|
||||
|
||||
@ -531,3 +531,31 @@ func (c *Client) UpdateSegmentStatistics(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) {
|
||||
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
|
||||
if !funcutil.CheckCtxValid(ctx) {
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
return client.(datapb.DataCoordClient).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) {
|
||||
ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) {
|
||||
if !funcutil.CheckCtxValid(ctx) {
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
return client.(datapb.DataCoordClient).ReleaseSegmentLock(ctx, req)
|
||||
})
|
||||
if err != nil || ret == nil {
|
||||
return nil, err
|
||||
}
|
||||
return ret.(*commonpb.Status), err
|
||||
}
|
||||
|
||||
@ -129,6 +129,12 @@ 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)
|
||||
}
|
||||
|
||||
client.grpcClient = &mock.GRPCClientBase{
|
||||
|
||||
@ -340,3 +340,13 @@ func (s *Server) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*da
|
||||
func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
|
||||
return s.dataCoord.UpdateSegmentStatistics(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)
|
||||
}
|
||||
|
||||
@ -58,6 +58,8 @@ type MockDataCoord struct {
|
||||
setSegmentStateResp *datapb.SetSegmentStateResponse
|
||||
importResp *datapb.ImportTaskResponse
|
||||
updateSegStatResp *commonpb.Status
|
||||
acquireSegLockResp *commonpb.Status
|
||||
releaseSegLockResp *commonpb.Status
|
||||
}
|
||||
|
||||
func (m *MockDataCoord) Init() error {
|
||||
@ -179,6 +181,14 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb
|
||||
return m.updateSegStatResp, 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 Test_NewServer(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
@ -420,6 +430,28 @@ 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)
|
||||
})
|
||||
|
||||
err := server.Stop()
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
@ -42,6 +42,9 @@ func TestIndexCoordClient(t *testing.T) {
|
||||
icm := &indexcoord.Mock{}
|
||||
err = server.SetClient(icm)
|
||||
assert.Nil(t, err)
|
||||
dcm := &indexcoord.DataCoordMock{}
|
||||
err = server.SetDataCoord(dcm)
|
||||
assert.Nil(t, err)
|
||||
|
||||
err = server.Run()
|
||||
assert.Nil(t, err)
|
||||
|
||||
@ -30,8 +30,9 @@ import (
|
||||
"google.golang.org/grpc/keepalive"
|
||||
|
||||
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
|
||||
|
||||
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
|
||||
"github.com/milvus-io/milvus/internal/indexcoord"
|
||||
ic "github.com/milvus-io/milvus/internal/indexcoord"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
@ -65,6 +66,8 @@ type Server struct {
|
||||
|
||||
etcdCli *clientv3.Client
|
||||
|
||||
dataCoord types.DataCoord
|
||||
|
||||
closer io.Closer
|
||||
}
|
||||
|
||||
@ -113,6 +116,34 @@ func (s *Server) init() error {
|
||||
return err
|
||||
}
|
||||
|
||||
// --- DataCoord ---
|
||||
if s.dataCoord == nil {
|
||||
s.dataCoord, err = dcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath, s.etcdCli)
|
||||
if err != nil {
|
||||
log.Debug("IndexCoord try to new DataCoord client failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
if err = s.dataCoord.Init(); err != nil {
|
||||
log.Debug("IndexCoord DataCoordClient Init failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
if err = s.dataCoord.Start(); err != nil {
|
||||
log.Debug("IndexCoord DataCoordClient Start failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
log.Debug("IndexCoord try to wait for DataCoord ready")
|
||||
err = funcutil.WaitForComponentHealthy(s.loopCtx, s.dataCoord, "DataCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
log.Debug("IndexCoord wait for DataCoord ready failed", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if err := s.SetDataCoord(s.dataCoord); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -160,6 +191,12 @@ func (s *Server) SetClient(indexCoordClient types.IndexCoordComponent) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetDataCoord sets the DataCoord's client for IndexCoord component.
|
||||
func (s *Server) SetDataCoord(d types.DataCoord) error {
|
||||
s.dataCoord = d
|
||||
return s.indexcoord.SetDataCoord(d)
|
||||
}
|
||||
|
||||
// GetComponentStates gets the component states of IndexCoord.
|
||||
func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) {
|
||||
return s.indexcoord.GetComponentStates(ctx)
|
||||
|
||||
@ -40,10 +40,14 @@ func TestIndexCoordinateServer(t *testing.T) {
|
||||
Params.Init()
|
||||
etcd, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, etcd)
|
||||
indexCoordClient := &indexcoord.Mock{}
|
||||
indexCoordClient.SetEtcdClient(etcd)
|
||||
err = server.SetClient(indexCoordClient)
|
||||
assert.Nil(t, err)
|
||||
|
||||
dcm := &indexcoord.DataCoordMock{}
|
||||
server.dataCoord = dcm
|
||||
err = server.Run()
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
||||
@ -518,6 +518,14 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *MockDataCoord) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
type MockProxy struct {
|
||||
MockBase
|
||||
|
||||
@ -161,12 +161,21 @@ func TestGrpcService(t *testing.T) {
|
||||
return nil
|
||||
}
|
||||
|
||||
core.CallGetBinlogFilePathsService = func(ctx context.Context, segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error) {
|
||||
return []string{"file1", "file2", "file3"}, nil
|
||||
}
|
||||
core.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error) {
|
||||
return rootcoord.Params.RootCoordCfg.MinSegmentSizeToEnableIndex, nil
|
||||
core.CallGetRecoveryInfoService = func(ctx context.Context, collID, partID rootcoord.UniqueID) ([]*datapb.SegmentBinlogs, error) {
|
||||
return []*datapb.SegmentBinlogs{
|
||||
{
|
||||
SegmentID: segID,
|
||||
NumOfRows: rootcoord.Params.RootCoordCfg.MinSegmentSizeToEnableIndex,
|
||||
FieldBinlogs: []*datapb.FieldBinlog{
|
||||
{
|
||||
FieldID: fieldID,
|
||||
Binlogs: []*datapb.Binlog{{LogPath: "file1"}, {LogPath: "file2"}, {LogPath: "file3"}},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
core.CallWatchChannels = func(ctx context.Context, collectionID int64, channelNames []string) error {
|
||||
return nil
|
||||
}
|
||||
@ -183,7 +192,7 @@ func TestGrpcService(t *testing.T) {
|
||||
|
||||
var binlogLock sync.Mutex
|
||||
binlogPathArray := make([]string, 0, 16)
|
||||
core.CallBuildIndexService = func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) {
|
||||
core.CallBuildIndexService = func(ctx context.Context, segID typeutil.UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) {
|
||||
binlogLock.Lock()
|
||||
defer binlogLock.Unlock()
|
||||
binlogPathArray = append(binlogPathArray, binlog...)
|
||||
|
||||
27
internal/indexcoord/const.go
Normal file
27
internal/indexcoord/const.go
Normal file
@ -0,0 +1,27 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package indexcoord
|
||||
|
||||
const (
|
||||
indexSizeFactor = 6
|
||||
indexFilePrefix = "indexes"
|
||||
)
|
||||
|
||||
const (
|
||||
// IndexAddTaskName is the name of the operation to add index task.
|
||||
IndexAddTaskName = "IndexAddTask"
|
||||
)
|
||||
@ -28,26 +28,25 @@ import (
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"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/metrics"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/tso"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"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"
|
||||
@ -60,10 +59,6 @@ import (
|
||||
// make sure IndexCoord implements types.IndexCoord
|
||||
var _ types.IndexCoord = (*IndexCoord)(nil)
|
||||
|
||||
const (
|
||||
indexSizeFactor = 6
|
||||
)
|
||||
|
||||
var Params paramtable.ComponentParam
|
||||
|
||||
// IndexCoord is a component responsible for scheduling index construction tasks and maintaining index status.
|
||||
@ -103,6 +98,8 @@ type IndexCoord struct {
|
||||
assignTaskInterval time.Duration
|
||||
taskLimit int
|
||||
|
||||
dataCoordClient types.DataCoord
|
||||
|
||||
// Add callback functions at different stages
|
||||
startCallbacks []func()
|
||||
closeCallbacks []func()
|
||||
@ -319,6 +316,16 @@ func (i *IndexCoord) SetEtcdClient(etcdClient *clientv3.Client) {
|
||||
i.etcdCli = etcdClient
|
||||
}
|
||||
|
||||
// SetDataCoord sets data coordinator's client
|
||||
func (i *IndexCoord) SetDataCoord(dataCoord types.DataCoord) error {
|
||||
if dataCoord == nil {
|
||||
return errors.New("null DataCoord interface")
|
||||
}
|
||||
|
||||
i.dataCoordClient = dataCoord
|
||||
return nil
|
||||
}
|
||||
|
||||
// UpdateStateCode updates the component state of IndexCoord.
|
||||
func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) {
|
||||
i.stateCode.Store(code)
|
||||
@ -395,21 +402,26 @@ func (i *IndexCoord) BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequ
|
||||
},
|
||||
}, err
|
||||
}
|
||||
metrics.IndexCoordIndexRequestCounter.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
log.Debug("IndexCoord building index ...",
|
||||
zap.Int64("IndexBuildID", req.IndexBuildID),
|
||||
zap.String("IndexName = ", req.IndexName),
|
||||
zap.Int64("IndexID = ", req.IndexID),
|
||||
zap.Strings("DataPath = ", req.DataPaths),
|
||||
zap.Any("TypeParams", req.TypeParams),
|
||||
zap.Any("IndexParams", req.IndexParams),
|
||||
zap.Int64("numRow", req.NumRows),
|
||||
log.Debug("IndexCoord building index ...", zap.Int64("segmentID", req.SegmentID),
|
||||
zap.String("IndexName", req.IndexName), zap.Int64("IndexID", req.IndexID),
|
||||
zap.Strings("DataPath", req.DataPaths), zap.Any("TypeParams", req.TypeParams),
|
||||
zap.Any("IndexParams", req.IndexParams), zap.Int64("numRows", req.NumRows),
|
||||
zap.Any("field type", req.FieldSchema.DataType))
|
||||
|
||||
ret := &indexpb.BuildIndexResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
IndexBuildID: 0,
|
||||
}
|
||||
|
||||
metrics.IndexCoordIndexRequestCounter.WithLabelValues(metrics.TotalLabel).Inc()
|
||||
|
||||
sp, ctx := trace.StartSpanFromContextWithOperationName(ctx, "IndexCoord-BuildIndex")
|
||||
defer sp.Finish()
|
||||
hasIndex, indexBuildID := i.metaTable.HasSameReq(req)
|
||||
if hasIndex {
|
||||
log.Debug("IndexCoord", zap.Int64("hasIndex true", indexBuildID), zap.Strings("data paths", req.DataPaths))
|
||||
log.Debug("IndexCoord has same index", zap.Int64("buildID", indexBuildID), zap.Int64("segmentID", req.SegmentID))
|
||||
return &indexpb.BuildIndexResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
@ -418,11 +430,7 @@ func (i *IndexCoord) BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequ
|
||||
IndexBuildID: indexBuildID,
|
||||
}, nil
|
||||
}
|
||||
ret := &indexpb.BuildIndexResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
}
|
||||
|
||||
t := &IndexAddTask{
|
||||
BaseTask: BaseTask{
|
||||
ctx: ctx,
|
||||
@ -837,6 +845,9 @@ func (i *IndexCoord) watchMetaLoop() {
|
||||
log.Debug("This task has finished", zap.Int64("indexBuildID", indexBuildID),
|
||||
zap.Int64("Finish by IndexNode", indexMeta.NodeID),
|
||||
zap.Int64("The version of the task", indexMeta.Version))
|
||||
if err = i.tryReleaseSegmentReferLock(ctx, []UniqueID{indexMeta.Req.SegmentID}); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
i.nodeManager.pq.IncPriority(indexMeta.NodeID, -1)
|
||||
metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.InProgressIndexTaskLabel).Dec()
|
||||
if indexMeta.State == commonpb.IndexState_Finished {
|
||||
@ -854,6 +865,47 @@ func (i *IndexCoord) watchMetaLoop() {
|
||||
}
|
||||
}
|
||||
|
||||
func (i *IndexCoord) tryAcquireSegmentReferLock(ctx context.Context, segIDs []UniqueID) error {
|
||||
status, err := i.dataCoordClient.AcquireSegmentLock(ctx, &datapb.AcquireSegmentLockRequest{
|
||||
NodeID: i.session.ServerID,
|
||||
SegmentIDs: segIDs,
|
||||
})
|
||||
if err != nil {
|
||||
log.Error("IndexCoord try to acquire segment reference lock failed", 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.Int64s("segIDs", segIDs),
|
||||
zap.Error(errors.New(status.Reason)))
|
||||
return errors.New(status.Reason)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, segIDs []UniqueID) error {
|
||||
releaseLock := func() error {
|
||||
status, err := i.dataCoordClient.ReleaseSegmentLock(ctx, &datapb.ReleaseSegmentLockRequest{
|
||||
NodeID: i.session.ServerID,
|
||||
SegmentIDs: segIDs,
|
||||
})
|
||||
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.Int64s("segIDs", segIDs),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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 (i *IndexCoord) assignTask(builderClient types.IndexNode, req *indexpb.CreateIndexRequest) bool {
|
||||
@ -903,6 +955,12 @@ func (i *IndexCoord) assignTaskLoop() {
|
||||
}
|
||||
for index, meta := range metas {
|
||||
indexBuildID := meta.indexMeta.IndexBuildID
|
||||
segID := meta.indexMeta.Req.SegmentID
|
||||
if err := i.tryAcquireSegmentReferLock(ctx, []UniqueID{segID}); err != nil {
|
||||
log.Warn("IndexCoord try to acquire segment reference lock failed, maybe this segment has been compacted",
|
||||
zap.Int64("segID", segID), zap.Int64("buildID", indexBuildID), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
if err := i.metaTable.UpdateVersion(indexBuildID); err != nil {
|
||||
log.Warn("IndexCoord assignmentTasksLoop metaTable.UpdateVersion failed", zap.Error(err))
|
||||
continue
|
||||
|
||||
@ -21,11 +21,15 @@ import (
|
||||
"errors"
|
||||
"strconv"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
@ -83,6 +87,10 @@ func (icm *Mock) SetEtcdClient(client *clientv3.Client) {
|
||||
icm.etcdCli = client
|
||||
}
|
||||
|
||||
func (icm *Mock) SetDataCoord(dataCoord types.DataCoord) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (icm *Mock) UpdateStateCode(stateCode internalpb.StateCode) {
|
||||
}
|
||||
|
||||
@ -243,3 +251,123 @@ func (icm *Mock) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsReq
|
||||
ComponentName: "IndexCoord",
|
||||
}, nil
|
||||
}
|
||||
|
||||
type DataCoordMock struct {
|
||||
types.DataCoord
|
||||
|
||||
Fail bool
|
||||
Err bool
|
||||
}
|
||||
|
||||
func (dcm *DataCoordMock) Init() error {
|
||||
if dcm.Err || dcm.Fail {
|
||||
return errors.New("DataCoord mock init failed")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dcm *DataCoordMock) Start() error {
|
||||
if dcm.Err || dcm.Fail {
|
||||
return errors.New("DataCoord mock start failed")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dcm *DataCoordMock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) {
|
||||
if dcm.Err {
|
||||
return &internalpb.ComponentStates{
|
||||
State: &internalpb.ComponentInfo{
|
||||
StateCode: internalpb.StateCode_Abnormal,
|
||||
},
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "",
|
||||
},
|
||||
}, errors.New("DataCoord component state is not healthy")
|
||||
}
|
||||
if dcm.Fail {
|
||||
return &internalpb.ComponentStates{
|
||||
State: &internalpb.ComponentInfo{
|
||||
StateCode: internalpb.StateCode_Abnormal,
|
||||
},
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "",
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
return &internalpb.ComponentStates{
|
||||
State: &internalpb.ComponentInfo{
|
||||
StateCode: internalpb.StateCode_Healthy,
|
||||
},
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (dcm *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||
if dcm.Err {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "",
|
||||
}, errors.New("an error occurred")
|
||||
}
|
||||
if dcm.Fail {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "failure reason",
|
||||
}, nil
|
||||
}
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (dcm *DataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
|
||||
if dcm.Err {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "",
|
||||
}, errors.New("an error occurred")
|
||||
}
|
||||
if dcm.Fail {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "failure reason",
|
||||
}, nil
|
||||
}
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
type ChunkManagerMock struct {
|
||||
storage.ChunkManager
|
||||
|
||||
Fail bool
|
||||
Err bool
|
||||
}
|
||||
|
||||
func (cmm *ChunkManagerMock) Exist(path string) (bool, error) {
|
||||
if cmm.Err {
|
||||
return false, errors.New("path not exist")
|
||||
}
|
||||
if cmm.Fail {
|
||||
return false, nil
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (cmm *ChunkManagerMock) RemoveWithPrefix(prefix string) error {
|
||||
if cmm.Err {
|
||||
return errors.New("error occurred")
|
||||
}
|
||||
if cmm.Fail {
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -20,6 +20,8 @@ import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
@ -201,3 +203,108 @@ func TestIndexCoordMockError(t *testing.T) {
|
||||
err = icm.Stop()
|
||||
assert.NotNil(t, err)
|
||||
}
|
||||
|
||||
func TestDataCoordMock_Error(t *testing.T) {
|
||||
t.Run("Init", func(t *testing.T) {
|
||||
dcm := &DataCoordMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
err := dcm.Init()
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("Start", func(t *testing.T) {
|
||||
dcm := &DataCoordMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
err := dcm.Start()
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("GetComponentStates", func(t *testing.T) {
|
||||
dcm := &DataCoordMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
resp, err := dcm.GetComponentStates(context.TODO())
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())
|
||||
|
||||
dcm.Err = false
|
||||
resp, err = dcm.GetComponentStates(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("AcquireSegmentLock", func(t *testing.T) {
|
||||
dcm := &DataCoordMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
resp, err := dcm.AcquireSegmentLock(context.TODO(), &datapb.AcquireSegmentLockRequest{})
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode())
|
||||
|
||||
dcm.Err = false
|
||||
resp, err = dcm.AcquireSegmentLock(context.TODO(), &datapb.AcquireSegmentLockRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("ReleaseSegmentLock", func(t *testing.T) {
|
||||
dcm := &DataCoordMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
|
||||
resp, err := dcm.ReleaseSegmentLock(context.TODO(), &datapb.ReleaseSegmentLockRequest{})
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode())
|
||||
|
||||
dcm.Err = false
|
||||
resp, err = dcm.ReleaseSegmentLock(context.TODO(), &datapb.ReleaseSegmentLockRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode())
|
||||
})
|
||||
}
|
||||
|
||||
func TestChunkManageMock_Error(t *testing.T) {
|
||||
t.Run("Exist", func(t *testing.T) {
|
||||
cmm := &ChunkManagerMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
ok, err := cmm.Exist("")
|
||||
assert.Error(t, err)
|
||||
assert.False(t, ok)
|
||||
|
||||
cmm.Err = false
|
||||
ok, err = cmm.Exist("")
|
||||
assert.NoError(t, err)
|
||||
assert.False(t, ok)
|
||||
|
||||
cmm.Fail = false
|
||||
ok, err = cmm.Exist("")
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, ok)
|
||||
})
|
||||
t.Run("RemoveWithPrefix", func(t *testing.T) {
|
||||
cmm := &ChunkManagerMock{
|
||||
Fail: true,
|
||||
Err: true,
|
||||
}
|
||||
|
||||
err := cmm.RemoveWithPrefix("")
|
||||
assert.Error(t, err)
|
||||
|
||||
cmm.Err = false
|
||||
err = cmm.RemoveWithPrefix("")
|
||||
assert.NoError(t, err)
|
||||
|
||||
cmm.Fail = false
|
||||
err = cmm.RemoveWithPrefix("")
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
@ -65,9 +65,23 @@ func TestIndexCoord(t *testing.T) {
|
||||
ic.assignTaskInterval = 200 * time.Millisecond
|
||||
ic.taskLimit = 20
|
||||
|
||||
dcm := &DataCoordMock{
|
||||
Err: false,
|
||||
Fail: false,
|
||||
}
|
||||
err = ic.SetDataCoord(dcm)
|
||||
assert.Nil(t, err)
|
||||
|
||||
ic.SetEtcdClient(etcdCli)
|
||||
err = ic.Init()
|
||||
assert.Nil(t, err)
|
||||
|
||||
ccm := &ChunkManagerMock{
|
||||
Err: false,
|
||||
Fail: false,
|
||||
}
|
||||
ic.chunkManager = ccm
|
||||
|
||||
err = ic.Register()
|
||||
assert.Nil(t, err)
|
||||
err = ic.Start()
|
||||
@ -387,4 +401,72 @@ func TestIndexCoord_GetIndexFilePaths(t *testing.T) {
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode)
|
||||
assert.NotEqual(t, "", resp.Status.Reason)
|
||||
})
|
||||
|
||||
t.Run("set DataCoord with nil", func(t *testing.T) {
|
||||
err := ic.SetDataCoord(nil)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_tryAcquireSegmentReferLock(t *testing.T) {
|
||||
ic := &IndexCoord{
|
||||
session: &sessionutil.Session{
|
||||
ServerID: 1,
|
||||
},
|
||||
}
|
||||
dcm := &DataCoordMock{
|
||||
Err: false,
|
||||
Fail: false,
|
||||
}
|
||||
cmm := &ChunkManagerMock{
|
||||
Err: false,
|
||||
Fail: false,
|
||||
}
|
||||
|
||||
ic.dataCoordClient = dcm
|
||||
ic.chunkManager = cmm
|
||||
|
||||
t.Run("success", func(t *testing.T) {
|
||||
err := ic.tryAcquireSegmentReferLock(context.Background(), []UniqueID{1})
|
||||
assert.Nil(t, err)
|
||||
})
|
||||
|
||||
t.Run("error", func(t *testing.T) {
|
||||
dcmE := &DataCoordMock{
|
||||
Err: true,
|
||||
Fail: false,
|
||||
}
|
||||
ic.dataCoordClient = dcmE
|
||||
err := ic.tryAcquireSegmentReferLock(context.Background(), []UniqueID{1})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("Fail", func(t *testing.T) {
|
||||
dcmF := &DataCoordMock{
|
||||
Err: false,
|
||||
Fail: true,
|
||||
}
|
||||
ic.dataCoordClient = dcmF
|
||||
err := ic.tryAcquireSegmentReferLock(context.Background(), []UniqueID{1})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_tryReleaseSegmentReferLock(t *testing.T) {
|
||||
ic := &IndexCoord{
|
||||
session: &sessionutil.Session{
|
||||
ServerID: 1,
|
||||
},
|
||||
}
|
||||
dcm := &DataCoordMock{
|
||||
Err: false,
|
||||
Fail: false,
|
||||
}
|
||||
|
||||
ic.dataCoordClient = dcm
|
||||
|
||||
t.Run("success", func(t *testing.T) {
|
||||
err := ic.tryReleaseSegmentReferLock(context.Background(), []UniqueID{1})
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
@ -20,6 +20,7 @@ import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"path"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
@ -99,7 +100,7 @@ func (mt *metaTable) saveIndexMeta(meta *Meta) error {
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
key := "indexes/" + strconv.FormatInt(meta.indexMeta.IndexBuildID, 10)
|
||||
key := path.Join(indexFilePrefix, strconv.FormatInt(meta.indexMeta.IndexBuildID, 10))
|
||||
err = mt.client.CompareVersionAndSwap(key, meta.revision, string(value))
|
||||
log.Debug("IndexCoord metaTable saveIndexMeta ", zap.String("key", key), zap.Error(err))
|
||||
if err != nil {
|
||||
@ -434,13 +435,13 @@ func (mt *metaTable) GetUnassignedTasks(onlineNodeIDs []int64) []Meta {
|
||||
|
||||
// HasSameReq determine whether there are same indexing tasks.
|
||||
func (mt *metaTable) HasSameReq(req *indexpb.BuildIndexRequest) (bool, UniqueID) {
|
||||
mt.lock.Lock()
|
||||
defer mt.lock.Unlock()
|
||||
mt.lock.RLock()
|
||||
defer mt.lock.RUnlock()
|
||||
|
||||
log.Debug("IndexCoord judges whether the same task exists in meta table", zap.Int64("indexBuildID", req.IndexBuildID),
|
||||
zap.Int64("indexID", req.IndexID), zap.Any("index params", req.IndexParams),
|
||||
zap.Any("type params", req.TypeParams))
|
||||
for _, meta := range mt.indexBuildID2Meta {
|
||||
if req.GetSegmentID() != meta.indexMeta.Req.GetSegmentID() {
|
||||
continue
|
||||
}
|
||||
if meta.indexMeta.Req.IndexID != req.IndexID {
|
||||
continue
|
||||
}
|
||||
@ -497,7 +498,7 @@ func (mt *metaTable) HasSameReq(req *indexpb.BuildIndexRequest) (bool, UniqueID)
|
||||
return true, meta.indexMeta.IndexBuildID
|
||||
}
|
||||
|
||||
return false, -1
|
||||
return false, 0
|
||||
}
|
||||
|
||||
// LoadMetaFromETCD load the meta of specified indexBuildID from ETCD.
|
||||
|
||||
@ -27,11 +27,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
)
|
||||
|
||||
const (
|
||||
// IndexAddTaskName is the name of the operation to add index task.
|
||||
IndexAddTaskName = "IndexAddTask"
|
||||
)
|
||||
|
||||
type task interface {
|
||||
Ctx() context.Context
|
||||
ID() UniqueID // return ReqID
|
||||
|
||||
@ -49,6 +49,9 @@ service DataCoord {
|
||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||
rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {}
|
||||
rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {}
|
||||
|
||||
rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {}
|
||||
rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {}
|
||||
}
|
||||
|
||||
service DataNode {
|
||||
@ -175,6 +178,17 @@ message GetPartitionStatisticsResponse {
|
||||
message GetSegmentInfoChannelRequest {
|
||||
}
|
||||
|
||||
message AcquireSegmentLockRequest {
|
||||
common.MsgBase base = 1;
|
||||
int64 nodeID = 2;
|
||||
repeated int64 segmentIDs = 3;
|
||||
}
|
||||
|
||||
message ReleaseSegmentLockRequest {
|
||||
common.MsgBase base = 1;
|
||||
int64 nodeID = 2;
|
||||
repeated int64 segmentIDs = 3;
|
||||
}
|
||||
|
||||
message VchannelInfo {
|
||||
int64 collectionID = 1;
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@ -80,6 +80,7 @@ message BuildIndexRequest {
|
||||
repeated common.KeyValuePair index_params = 7;
|
||||
int64 num_rows = 8;
|
||||
schema.FieldSchema field_schema = 9;
|
||||
int64 segmentID = 10;
|
||||
}
|
||||
|
||||
message BuildIndexResponse {
|
||||
|
||||
@ -391,6 +391,7 @@ type BuildIndexRequest struct {
|
||||
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,7,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||
NumRows int64 `protobuf:"varint,8,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
|
||||
FieldSchema *schemapb.FieldSchema `protobuf:"bytes,9,opt,name=field_schema,json=fieldSchema,proto3" json:"field_schema,omitempty"`
|
||||
SegmentID int64 `protobuf:"varint,10,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
@ -477,6 +478,13 @@ func (m *BuildIndexRequest) GetFieldSchema() *schemapb.FieldSchema {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *BuildIndexRequest) GetSegmentID() int64 {
|
||||
if m != nil {
|
||||
return m.SegmentID
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type BuildIndexResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
IndexBuildID int64 `protobuf:"varint,2,opt,name=indexBuildID,proto3" json:"indexBuildID,omitempty"`
|
||||
@ -842,75 +850,75 @@ func init() {
|
||||
func init() { proto.RegisterFile("index_coord.proto", fileDescriptor_f9e019eb3fda53c2) }
|
||||
|
||||
var fileDescriptor_f9e019eb3fda53c2 = []byte{
|
||||
// 1073 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0xcd, 0x6e, 0xdb, 0x46,
|
||||
0x10, 0x36, 0x4d, 0x5b, 0x3f, 0x23, 0xc5, 0x8d, 0xb7, 0x69, 0xc0, 0x28, 0x0d, 0x22, 0xb3, 0xf9,
|
||||
0x51, 0x8b, 0x44, 0x0e, 0x94, 0xa6, 0x3d, 0x15, 0x68, 0x6d, 0x21, 0x86, 0x51, 0x38, 0x30, 0xd6,
|
||||
0x46, 0x0f, 0x05, 0x0a, 0x61, 0x2d, 0x8e, 0xec, 0x45, 0xf8, 0x23, 0x73, 0x57, 0x49, 0xed, 0x63,
|
||||
0xd1, 0x7b, 0x6f, 0xe9, 0xa3, 0xf4, 0xd8, 0x67, 0xc8, 0xe3, 0xf4, 0x56, 0xec, 0x72, 0x29, 0x91,
|
||||
0x12, 0xe5, 0xc8, 0x75, 0xd3, 0x5e, 0x72, 0xe3, 0xcc, 0x7e, 0xb3, 0xb3, 0xf3, 0xed, 0xec, 0xc7,
|
||||
0x81, 0x75, 0x1e, 0x7a, 0xf8, 0x73, 0xaf, 0x1f, 0x45, 0xb1, 0xd7, 0x1e, 0xc6, 0x91, 0x8c, 0x08,
|
||||
0x09, 0xb8, 0xff, 0x6a, 0x24, 0x12, 0xab, 0xad, 0xd7, 0x1b, 0xf5, 0x7e, 0x14, 0x04, 0x51, 0x98,
|
||||
0xf8, 0x1a, 0x6b, 0x3c, 0x94, 0x18, 0x87, 0xcc, 0x37, 0x76, 0x3d, 0x1b, 0xd1, 0xa8, 0x8b, 0xfe,
|
||||
0x09, 0x06, 0x2c, 0xb1, 0xdc, 0xdf, 0x2d, 0xf8, 0x98, 0xe2, 0x31, 0x17, 0x12, 0xe3, 0x17, 0x91,
|
||||
0x87, 0x14, 0x4f, 0x47, 0x28, 0x24, 0x79, 0x02, 0x2b, 0x47, 0x4c, 0xa0, 0x63, 0x35, 0xad, 0x56,
|
||||
0xad, 0xf3, 0x69, 0x3b, 0x97, 0xd4, 0x64, 0xdb, 0x13, 0xc7, 0x5b, 0x4c, 0x20, 0xd5, 0x48, 0xf2,
|
||||
0x15, 0x94, 0x99, 0xe7, 0xc5, 0x28, 0x84, 0xb3, 0x7c, 0x41, 0xd0, 0x77, 0x09, 0x86, 0xa6, 0x60,
|
||||
0x72, 0x13, 0x4a, 0x61, 0xe4, 0xe1, 0x6e, 0xd7, 0xb1, 0x9b, 0x56, 0xcb, 0xa6, 0xc6, 0x72, 0x7f,
|
||||
0xb3, 0xe0, 0x46, 0xfe, 0x64, 0x62, 0x18, 0x85, 0x02, 0xc9, 0x53, 0x28, 0x09, 0xc9, 0xe4, 0x48,
|
||||
0x98, 0xc3, 0xdd, 0x2e, 0xcc, 0x73, 0xa0, 0x21, 0xd4, 0x40, 0xc9, 0x16, 0xd4, 0x78, 0xc8, 0x65,
|
||||
0x6f, 0xc8, 0x62, 0x16, 0xa4, 0x27, 0xdc, 0x68, 0x4f, 0x71, 0x69, 0x68, 0xdb, 0x0d, 0xb9, 0xdc,
|
||||
0xd7, 0x40, 0x0a, 0x7c, 0xfc, 0xed, 0x7e, 0x03, 0x9f, 0xec, 0xa0, 0xdc, 0x55, 0x8c, 0xab, 0xdd,
|
||||
0x51, 0xa4, 0x64, 0xdd, 0x83, 0x6b, 0xfa, 0x1e, 0xb6, 0x46, 0xdc, 0xf7, 0x76, 0xbb, 0xea, 0x60,
|
||||
0x76, 0xcb, 0xa6, 0x79, 0xa7, 0xfb, 0x87, 0x05, 0x55, 0x1d, 0xbc, 0x1b, 0x0e, 0x22, 0xf2, 0x0c,
|
||||
0x56, 0xd5, 0xd1, 0x12, 0x86, 0xd7, 0x3a, 0x77, 0x0b, 0x8b, 0x98, 0xe4, 0xa2, 0x09, 0x9a, 0xb8,
|
||||
0x50, 0xcf, 0xee, 0xaa, 0x0b, 0xb1, 0x69, 0xce, 0x47, 0x1c, 0x28, 0x6b, 0x7b, 0x4c, 0x69, 0x6a,
|
||||
0x92, 0x3b, 0x00, 0x49, 0x43, 0x85, 0x2c, 0x40, 0x67, 0xa5, 0x69, 0xb5, 0xaa, 0xb4, 0xaa, 0x3d,
|
||||
0x2f, 0x58, 0x80, 0xea, 0x2a, 0x62, 0x64, 0x22, 0x0a, 0x9d, 0x55, 0xbd, 0x64, 0x2c, 0xf7, 0x57,
|
||||
0x0b, 0x6e, 0x4e, 0x57, 0x7e, 0x95, 0xcb, 0x78, 0x96, 0x04, 0xa1, 0xba, 0x07, 0xbb, 0x55, 0xeb,
|
||||
0xdc, 0x69, 0xcf, 0xf6, 0x74, 0x7b, 0x4c, 0x15, 0x35, 0x60, 0xf7, 0xed, 0x32, 0x90, 0xed, 0x18,
|
||||
0x99, 0x44, 0xbd, 0x96, 0xb2, 0x3f, 0x4d, 0x89, 0x55, 0x40, 0x49, 0xbe, 0xf0, 0xe5, 0xe9, 0xc2,
|
||||
0xe7, 0x33, 0xe6, 0x40, 0xf9, 0x15, 0xc6, 0x82, 0x47, 0xa1, 0xa6, 0xcb, 0xa6, 0xa9, 0x49, 0x6e,
|
||||
0x43, 0x35, 0x40, 0xc9, 0x7a, 0x43, 0x26, 0x4f, 0x0c, 0x5f, 0x15, 0xe5, 0xd8, 0x67, 0xf2, 0x44,
|
||||
0xe5, 0xf3, 0x98, 0x59, 0x14, 0x4e, 0xa9, 0x69, 0xab, 0x7c, 0xca, 0xa3, 0x56, 0x75, 0x37, 0xca,
|
||||
0xb3, 0x21, 0xa6, 0xdd, 0x58, 0xd6, 0x2c, 0x6c, 0x14, 0x52, 0xf7, 0x3d, 0x9e, 0xfd, 0xc0, 0xfc,
|
||||
0x11, 0xee, 0x33, 0x1e, 0x53, 0x50, 0x51, 0x49, 0x37, 0x92, 0xae, 0x29, 0x3b, 0xdd, 0xa4, 0xb2,
|
||||
0xe8, 0x26, 0x35, 0x1d, 0x66, 0x7a, 0xfa, 0xaf, 0x65, 0x58, 0x4f, 0x48, 0xfa, 0xcf, 0x28, 0xcd,
|
||||
0x73, 0xb3, 0xfa, 0x0e, 0x6e, 0x4a, 0xff, 0x06, 0x37, 0xe5, 0x7f, 0xc2, 0x0d, 0xb9, 0x05, 0x95,
|
||||
0x70, 0x14, 0xf4, 0xe2, 0xe8, 0xb5, 0x62, 0x57, 0xd7, 0x10, 0x8e, 0x02, 0x1a, 0xbd, 0x16, 0x64,
|
||||
0x1b, 0xea, 0x03, 0x8e, 0xbe, 0xd7, 0x4b, 0xc4, 0xd4, 0xa9, 0xea, 0xe6, 0x6f, 0xe6, 0x13, 0x18,
|
||||
0xa1, 0x7d, 0xae, 0x80, 0x07, 0xfa, 0x9b, 0xd6, 0x06, 0x13, 0xc3, 0x0d, 0x80, 0x64, 0xa9, 0xbf,
|
||||
0xca, 0x8b, 0x5a, 0x40, 0x16, 0xdc, 0x6f, 0xc1, 0x49, 0x1f, 0xf1, 0x73, 0xee, 0xa3, 0x66, 0xfb,
|
||||
0x72, 0x0a, 0xf6, 0xa7, 0x05, 0xeb, 0xb9, 0x78, 0xad, 0x64, 0xef, 0xeb, 0xc0, 0xa4, 0x05, 0xd7,
|
||||
0x93, 0x5b, 0x1c, 0x70, 0x1f, 0x4d, 0xbb, 0xd8, 0xba, 0x5d, 0xd6, 0x78, 0xae, 0x0a, 0xf2, 0x10,
|
||||
0x3e, 0x12, 0x18, 0x73, 0xe6, 0xf3, 0x73, 0xf4, 0x7a, 0x82, 0x9f, 0x27, 0xe2, 0xb6, 0x42, 0xd7,
|
||||
0x26, 0xee, 0x03, 0x7e, 0x8e, 0xee, 0x1b, 0x0b, 0x6e, 0x15, 0x90, 0x70, 0x15, 0xea, 0xbb, 0x00,
|
||||
0x99, 0xf3, 0x25, 0x82, 0x76, 0x7f, 0xae, 0xa0, 0x65, 0x99, 0xa3, 0xd5, 0x41, 0x7a, 0x04, 0xf7,
|
||||
0x17, 0xdb, 0xfc, 0x1c, 0xf6, 0x50, 0xb2, 0x85, 0xde, 0xdf, 0xf8, 0x07, 0xb2, 0x7c, 0xa9, 0x1f,
|
||||
0xc8, 0x5d, 0xa8, 0x0d, 0x18, 0xf7, 0x7b, 0x46, 0xe8, 0x6d, 0xfd, 0x6e, 0x41, 0xb9, 0xa8, 0xf6,
|
||||
0x90, 0xaf, 0xc1, 0x8e, 0xf1, 0x54, 0xf3, 0x37, 0xa7, 0x90, 0x19, 0xbd, 0xa0, 0x2a, 0xa2, 0xf0,
|
||||
0xba, 0x56, 0x0b, 0xaf, 0x6b, 0x03, 0xea, 0x01, 0x8b, 0x5f, 0xf6, 0x3c, 0xf4, 0x51, 0xa2, 0xe7,
|
||||
0x94, 0x9a, 0x56, 0xab, 0x42, 0x6b, 0xca, 0xd7, 0x4d, 0x5c, 0x99, 0xa9, 0xa0, 0x9c, 0x9d, 0x0a,
|
||||
0xb2, 0x7a, 0x5c, 0xc9, 0xeb, 0x71, 0x03, 0x2a, 0x31, 0xf6, 0xcf, 0xfa, 0x3e, 0x7a, 0xfa, 0x39,
|
||||
0x56, 0xe8, 0xd8, 0x26, 0xf7, 0x61, 0xd2, 0x08, 0x49, 0x7b, 0x80, 0x6e, 0x8f, 0x6b, 0x63, 0xaf,
|
||||
0xee, 0x8e, 0x47, 0x70, 0xbd, 0x1b, 0x47, 0xc3, 0x9c, 0x14, 0x66, 0x74, 0xcc, 0xca, 0xe9, 0x58,
|
||||
0xe7, 0x6d, 0x09, 0x40, 0x43, 0xb7, 0xd5, 0x74, 0x46, 0x86, 0x40, 0x76, 0x50, 0x6e, 0x47, 0xc1,
|
||||
0x30, 0x0a, 0x31, 0x94, 0xc9, 0x7f, 0x92, 0x3c, 0x99, 0x33, 0x62, 0xcc, 0x42, 0x4d, 0xc2, 0xc6,
|
||||
0x83, 0x39, 0x11, 0x53, 0x70, 0x77, 0x89, 0x04, 0x3a, 0xe3, 0x21, 0x0f, 0xf0, 0x90, 0xf7, 0x5f,
|
||||
0x6e, 0x9f, 0xb0, 0x30, 0x44, 0xff, 0xa2, 0x8c, 0x53, 0xd0, 0x34, 0xe3, 0x67, 0xf9, 0x08, 0x63,
|
||||
0x1c, 0xc8, 0x98, 0x87, 0xc7, 0xe9, 0xdb, 0x70, 0x97, 0xc8, 0x29, 0xdc, 0xd8, 0x41, 0x9d, 0x9d,
|
||||
0x0b, 0xc9, 0xfb, 0x22, 0x4d, 0xd8, 0x99, 0x9f, 0x70, 0x06, 0x7c, 0xc9, 0x94, 0x3f, 0x01, 0x4c,
|
||||
0x9a, 0x8d, 0x2c, 0xd6, 0x8c, 0xb3, 0x04, 0x4e, 0xc3, 0xc6, 0xdb, 0x73, 0x58, 0xcb, 0x8f, 0x35,
|
||||
0xe4, 0xf3, 0xa2, 0xd8, 0xc2, 0xa1, 0xaf, 0xf1, 0xc5, 0x22, 0xd0, 0x71, 0xaa, 0x18, 0xd6, 0x67,
|
||||
0x74, 0x87, 0x3c, 0xba, 0x68, 0x8b, 0x69, 0x8d, 0x6e, 0x3c, 0x5e, 0x10, 0x3d, 0xce, 0xb9, 0x0f,
|
||||
0xd5, 0x71, 0x3b, 0x93, 0x7b, 0x45, 0xd1, 0xd3, 0xdd, 0xde, 0xb8, 0x48, 0xf1, 0xdc, 0x25, 0xd2,
|
||||
0x03, 0xd8, 0x41, 0xb9, 0x87, 0x32, 0xe6, 0x7d, 0x41, 0x1e, 0x14, 0x5e, 0xe2, 0x04, 0x90, 0x6e,
|
||||
0xfa, 0xf0, 0x9d, 0xb8, 0xf4, 0xc8, 0x9d, 0x37, 0x2b, 0x46, 0x06, 0xd5, 0xc4, 0xff, 0xe1, 0x49,
|
||||
0xbd, 0x87, 0x27, 0x75, 0x08, 0xb5, 0xcc, 0x0c, 0x4d, 0x0a, 0x1f, 0xcb, 0xec, 0x90, 0xfd, 0x7f,
|
||||
0x37, 0xc6, 0xd6, 0x97, 0x3f, 0x76, 0x8e, 0xb9, 0x3c, 0x19, 0x1d, 0xa9, 0xd4, 0x9b, 0x09, 0xf2,
|
||||
0x31, 0x8f, 0xcc, 0xd7, 0x66, 0xca, 0xd0, 0xa6, 0xde, 0x69, 0x53, 0x97, 0x31, 0x3c, 0x3a, 0x2a,
|
||||
0x69, 0xf3, 0xe9, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xd8, 0x29, 0x0d, 0x3e, 0x47, 0x0f, 0x00,
|
||||
0x00,
|
||||
// 1087 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x57, 0xcd, 0x6e, 0x1b, 0x37,
|
||||
0x10, 0xf6, 0x7a, 0x6d, 0xfd, 0x8c, 0x14, 0x37, 0x66, 0xd3, 0x60, 0xa3, 0x24, 0x88, 0xbc, 0xcd,
|
||||
0x8f, 0x5a, 0x24, 0x72, 0xa0, 0x34, 0xed, 0xa9, 0x40, 0x6b, 0x0b, 0x31, 0x84, 0xc2, 0x81, 0x41,
|
||||
0x1b, 0x3d, 0x14, 0x28, 0x04, 0x5a, 0x3b, 0xb2, 0x88, 0xec, 0x8f, 0xbc, 0xa4, 0x92, 0xda, 0xc7,
|
||||
0xa2, 0xf7, 0xde, 0x92, 0x47, 0xe9, 0xb1, 0xcf, 0x90, 0x37, 0x2a, 0xc8, 0xe5, 0x4a, 0x5a, 0x69,
|
||||
0xe5, 0xc8, 0x75, 0xd3, 0x5e, 0x7a, 0xd3, 0xcc, 0x7e, 0xc3, 0x8f, 0xf3, 0xcd, 0x70, 0x48, 0xc1,
|
||||
0x26, 0x0f, 0x3d, 0xfc, 0xa5, 0xdb, 0x8b, 0xa2, 0xd8, 0x6b, 0x0e, 0xe3, 0x48, 0x46, 0x84, 0x04,
|
||||
0xdc, 0x7f, 0x3d, 0x12, 0x89, 0xd5, 0xd4, 0xdf, 0x6b, 0xd5, 0x5e, 0x14, 0x04, 0x51, 0x98, 0xf8,
|
||||
0x6a, 0x1b, 0x3c, 0x94, 0x18, 0x87, 0xcc, 0x37, 0x76, 0x75, 0x3a, 0xa2, 0x56, 0x15, 0xbd, 0x01,
|
||||
0x06, 0x2c, 0xb1, 0xdc, 0x77, 0x16, 0x7c, 0x4a, 0xf1, 0x84, 0x0b, 0x89, 0xf1, 0xcb, 0xc8, 0x43,
|
||||
0x8a, 0xa7, 0x23, 0x14, 0x92, 0x3c, 0x85, 0xb5, 0x63, 0x26, 0xd0, 0xb1, 0xea, 0x56, 0xa3, 0xd2,
|
||||
0xba, 0xd3, 0xcc, 0x90, 0x1a, 0xb6, 0x7d, 0x71, 0xb2, 0xc3, 0x04, 0x52, 0x8d, 0x24, 0x5f, 0x43,
|
||||
0x91, 0x79, 0x5e, 0x8c, 0x42, 0x38, 0xab, 0x17, 0x04, 0x7d, 0x9f, 0x60, 0x68, 0x0a, 0x26, 0x37,
|
||||
0xa1, 0x10, 0x46, 0x1e, 0x76, 0xda, 0x8e, 0x5d, 0xb7, 0x1a, 0x36, 0x35, 0x96, 0xfb, 0xbb, 0x05,
|
||||
0x37, 0xb2, 0x3b, 0x13, 0xc3, 0x28, 0x14, 0x48, 0x9e, 0x41, 0x41, 0x48, 0x26, 0x47, 0xc2, 0x6c,
|
||||
0xee, 0x76, 0x2e, 0xcf, 0xa1, 0x86, 0x50, 0x03, 0x25, 0x3b, 0x50, 0xe1, 0x21, 0x97, 0xdd, 0x21,
|
||||
0x8b, 0x59, 0x90, 0xee, 0x70, 0xab, 0x39, 0xa3, 0xa5, 0x91, 0xad, 0x13, 0x72, 0x79, 0xa0, 0x81,
|
||||
0x14, 0xf8, 0xf8, 0xb7, 0xfb, 0x2d, 0x7c, 0xb6, 0x87, 0xb2, 0xa3, 0x14, 0x57, 0xab, 0xa3, 0x48,
|
||||
0xc5, 0xba, 0x0f, 0xd7, 0x74, 0x1d, 0x76, 0x46, 0xdc, 0xf7, 0x3a, 0x6d, 0xb5, 0x31, 0xbb, 0x61,
|
||||
0xd3, 0xac, 0xd3, 0xfd, 0xc3, 0x82, 0xb2, 0x0e, 0xee, 0x84, 0xfd, 0x88, 0x3c, 0x87, 0x75, 0xb5,
|
||||
0xb5, 0x44, 0xe1, 0x8d, 0xd6, 0xbd, 0xdc, 0x24, 0x26, 0x5c, 0x34, 0x41, 0x13, 0x17, 0xaa, 0xd3,
|
||||
0xab, 0xea, 0x44, 0x6c, 0x9a, 0xf1, 0x11, 0x07, 0x8a, 0xda, 0x1e, 0x4b, 0x9a, 0x9a, 0xe4, 0x2e,
|
||||
0x40, 0xd2, 0x50, 0x21, 0x0b, 0xd0, 0x59, 0xab, 0x5b, 0x8d, 0x32, 0x2d, 0x6b, 0xcf, 0x4b, 0x16,
|
||||
0xa0, 0x2a, 0x45, 0x8c, 0x4c, 0x44, 0xa1, 0xb3, 0xae, 0x3f, 0x19, 0xcb, 0xfd, 0xcd, 0x82, 0x9b,
|
||||
0xb3, 0x99, 0x5f, 0xa5, 0x18, 0xcf, 0x93, 0x20, 0x54, 0x75, 0xb0, 0x1b, 0x95, 0xd6, 0xdd, 0xe6,
|
||||
0x7c, 0x4f, 0x37, 0xc7, 0x52, 0x51, 0x03, 0x76, 0xdf, 0xaf, 0x02, 0xd9, 0x8d, 0x91, 0x49, 0xd4,
|
||||
0xdf, 0x52, 0xf5, 0x67, 0x25, 0xb1, 0x72, 0x24, 0xc9, 0x26, 0xbe, 0x3a, 0x9b, 0xf8, 0x62, 0xc5,
|
||||
0x1c, 0x28, 0xbe, 0xc6, 0x58, 0xf0, 0x28, 0xd4, 0x72, 0xd9, 0x34, 0x35, 0xc9, 0x6d, 0x28, 0x07,
|
||||
0x28, 0x59, 0x77, 0xc8, 0xe4, 0xc0, 0xe8, 0x55, 0x52, 0x8e, 0x03, 0x26, 0x07, 0x8a, 0xcf, 0x63,
|
||||
0xe6, 0xa3, 0x70, 0x0a, 0x75, 0x5b, 0xf1, 0x29, 0x8f, 0xfa, 0xaa, 0xbb, 0x51, 0x9e, 0x0d, 0x31,
|
||||
0xed, 0xc6, 0xa2, 0x56, 0x61, 0x2b, 0x57, 0xba, 0x1f, 0xf0, 0xec, 0x47, 0xe6, 0x8f, 0xf0, 0x80,
|
||||
0xf1, 0x98, 0x82, 0x8a, 0x4a, 0xba, 0x91, 0xb4, 0x4d, 0xda, 0xe9, 0x22, 0xa5, 0x65, 0x17, 0xa9,
|
||||
0xe8, 0x30, 0xd3, 0xd3, 0xef, 0x6c, 0xd8, 0x4c, 0x44, 0xfa, 0xd7, 0x24, 0xcd, 0x6a, 0xb3, 0xfe,
|
||||
0x01, 0x6d, 0x0a, 0xff, 0x84, 0x36, 0xc5, 0xbf, 0xa3, 0x0d, 0xb9, 0x05, 0xa5, 0x70, 0x14, 0x74,
|
||||
0xe3, 0xe8, 0x8d, 0x52, 0x57, 0xe7, 0x10, 0x8e, 0x02, 0x1a, 0xbd, 0x11, 0x64, 0x17, 0xaa, 0x7d,
|
||||
0x8e, 0xbe, 0xd7, 0x4d, 0x86, 0xa9, 0x53, 0xd6, 0xcd, 0x5f, 0xcf, 0x12, 0x98, 0x41, 0xfb, 0x42,
|
||||
0x01, 0x0f, 0xf5, 0x6f, 0x5a, 0xe9, 0x4f, 0x0c, 0x72, 0x07, 0xca, 0x02, 0x4f, 0x02, 0x0c, 0x65,
|
||||
0xa7, 0xed, 0x80, 0x26, 0x98, 0x38, 0xdc, 0x00, 0xc8, 0x74, 0x61, 0xae, 0x72, 0xde, 0x96, 0x18,
|
||||
0x1a, 0xee, 0x77, 0xe0, 0xa4, 0x47, 0xfc, 0x05, 0xf7, 0x51, 0xd7, 0xe2, 0x72, 0xf3, 0xed, 0x4f,
|
||||
0x0b, 0x36, 0x33, 0xf1, 0x7a, 0xce, 0x7d, 0xac, 0x0d, 0x93, 0x06, 0x5c, 0x4f, 0x6a, 0xdc, 0xe7,
|
||||
0x3e, 0x9a, 0x66, 0xb2, 0x75, 0x33, 0x6d, 0xf0, 0x4c, 0x16, 0xe4, 0x11, 0x7c, 0x22, 0x30, 0xe6,
|
||||
0xcc, 0xe7, 0xe7, 0xe8, 0x75, 0x05, 0x3f, 0x4f, 0x46, 0xdf, 0x1a, 0xdd, 0x98, 0xb8, 0x0f, 0xf9,
|
||||
0x39, 0xba, 0x6f, 0x2d, 0xb8, 0x95, 0x23, 0xc2, 0x55, 0xa4, 0x6f, 0x03, 0x4c, 0xed, 0x2f, 0x19,
|
||||
0x77, 0x0f, 0x16, 0x8e, 0xbb, 0x69, 0xe5, 0x68, 0xb9, 0x9f, 0x6e, 0xc1, 0xfd, 0xd5, 0x36, 0x57,
|
||||
0xc7, 0x3e, 0x4a, 0xb6, 0xd4, 0xe9, 0x1c, 0x5f, 0x2f, 0xab, 0x97, 0xba, 0x5e, 0xee, 0x41, 0xa5,
|
||||
0xcf, 0xb8, 0xdf, 0x35, 0xd7, 0x80, 0xad, 0x4f, 0x35, 0x28, 0x17, 0xd5, 0x1e, 0xf2, 0x0d, 0xd8,
|
||||
0x31, 0x9e, 0x6a, 0xfd, 0x16, 0x24, 0x32, 0x37, 0x4d, 0xa8, 0x8a, 0xc8, 0x2d, 0xd7, 0x7a, 0x6e,
|
||||
0xb9, 0xb6, 0xa0, 0x1a, 0xb0, 0xf8, 0x55, 0xd7, 0x43, 0x1f, 0x25, 0x7a, 0x4e, 0xa1, 0x6e, 0x35,
|
||||
0x4a, 0xb4, 0xa2, 0x7c, 0xed, 0xc4, 0x35, 0xf5, 0x66, 0x28, 0x4e, 0xbf, 0x19, 0xa6, 0xa7, 0x75,
|
||||
0x29, 0x3b, 0xad, 0x6b, 0x50, 0x8a, 0xb1, 0x77, 0xd6, 0xf3, 0xd1, 0xd3, 0x87, 0xb5, 0x44, 0xc7,
|
||||
0x36, 0x79, 0x00, 0x93, 0x46, 0x48, 0xda, 0x03, 0x74, 0x7b, 0x5c, 0x1b, 0x7b, 0x75, 0x77, 0x3c,
|
||||
0x86, 0xeb, 0xed, 0x38, 0x1a, 0x66, 0x06, 0xe5, 0xd4, 0x94, 0xb3, 0x32, 0x53, 0xae, 0xf5, 0xbe,
|
||||
0x00, 0xa0, 0xa1, 0xbb, 0xea, 0xed, 0x46, 0x86, 0x40, 0xf6, 0x50, 0xee, 0x46, 0xc1, 0x30, 0x0a,
|
||||
0x31, 0x94, 0xc9, 0x2d, 0x4a, 0x9e, 0x2e, 0x78, 0x80, 0xcc, 0x43, 0x0d, 0x61, 0xed, 0xe1, 0x82,
|
||||
0x88, 0x19, 0xb8, 0xbb, 0x42, 0x02, 0xcd, 0x78, 0xc4, 0x03, 0x3c, 0xe2, 0xbd, 0x57, 0xbb, 0x03,
|
||||
0x16, 0x86, 0xe8, 0x5f, 0xc4, 0x38, 0x03, 0x4d, 0x19, 0x3f, 0xcf, 0x46, 0x18, 0xe3, 0x50, 0xc6,
|
||||
0x3c, 0x3c, 0x49, 0xcf, 0x86, 0xbb, 0x42, 0x4e, 0xe1, 0xc6, 0x1e, 0x6a, 0x76, 0x2e, 0x24, 0xef,
|
||||
0x89, 0x94, 0xb0, 0xb5, 0x98, 0x70, 0x0e, 0x7c, 0x49, 0xca, 0x9f, 0x01, 0x26, 0xcd, 0x46, 0x96,
|
||||
0x6b, 0xc6, 0x79, 0x01, 0x67, 0x61, 0xe3, 0xe5, 0x39, 0x6c, 0x64, 0x1f, 0x3d, 0xe4, 0x8b, 0xbc,
|
||||
0xd8, 0xdc, 0x27, 0x61, 0xed, 0xcb, 0x65, 0xa0, 0x63, 0xaa, 0x18, 0x36, 0xe7, 0xe6, 0x0e, 0x79,
|
||||
0x7c, 0xd1, 0x12, 0xb3, 0x33, 0xba, 0xf6, 0x64, 0x49, 0xf4, 0x98, 0xf3, 0x00, 0xca, 0xe3, 0x76,
|
||||
0x26, 0xf7, 0xf3, 0xa2, 0x67, 0xbb, 0xbd, 0x76, 0xd1, 0xc4, 0x73, 0x57, 0x48, 0x17, 0x60, 0x0f,
|
||||
0xe5, 0x3e, 0xca, 0x98, 0xf7, 0x04, 0x79, 0x98, 0x5b, 0xc4, 0x09, 0x20, 0x5d, 0xf4, 0xd1, 0x07,
|
||||
0x71, 0xe9, 0x96, 0x5b, 0x6f, 0xd7, 0xcc, 0x18, 0x54, 0xff, 0x07, 0xfe, 0x3f, 0x52, 0x1f, 0xe1,
|
||||
0x48, 0x1d, 0x41, 0x65, 0xea, 0x85, 0x4d, 0x72, 0x0f, 0xcb, 0xfc, 0x13, 0xfc, 0xbf, 0x6e, 0x8c,
|
||||
0x9d, 0xaf, 0x7e, 0x6a, 0x9d, 0x70, 0x39, 0x18, 0x1d, 0x2b, 0xea, 0xed, 0x04, 0xf9, 0x84, 0x47,
|
||||
0xe6, 0xd7, 0x76, 0xaa, 0xd0, 0xb6, 0x5e, 0x69, 0x5b, 0xa7, 0x31, 0x3c, 0x3e, 0x2e, 0x68, 0xf3,
|
||||
0xd9, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x71, 0x6a, 0x92, 0xdd, 0x65, 0x0f, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
|
||||
@ -221,6 +221,20 @@ func (coord *DataCoordMock) UpdateSegmentStatistics(ctx context.Context, req *da
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (coord *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*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) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func NewDataCoordMock() *DataCoordMock {
|
||||
return &DataCoordMock{
|
||||
nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
|
||||
|
||||
@ -6,6 +6,8 @@ import (
|
||||
"fmt"
|
||||
"path"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/retry"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
|
||||
"go.uber.org/zap"
|
||||
@ -476,3 +478,55 @@ func (broker *globalMetaBroker) getSegmentStates(ctx context.Context, segmentID
|
||||
|
||||
return resp.States[0], nil
|
||||
}
|
||||
|
||||
func (broker *globalMetaBroker) acquireSegmentsReferLock(ctx context.Context, segmentIDs []UniqueID) error {
|
||||
ctx, cancel := context.WithTimeout(ctx, timeoutForRPC)
|
||||
defer cancel()
|
||||
acquireSegLockReq := &datapb.AcquireSegmentLockRequest{
|
||||
SegmentIDs: segmentIDs,
|
||||
NodeID: Params.QueryCoordCfg.GetNodeID(),
|
||||
}
|
||||
status, err := broker.dataCoord.AcquireSegmentLock(ctx, acquireSegLockReq)
|
||||
if err != nil {
|
||||
log.Error("QueryCoord acquire the segment reference lock error", zap.Int64s("segIDs", segmentIDs),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
if status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
log.Error("QueryCoord acquire the segment reference lock error", zap.Int64s("segIDs", segmentIDs),
|
||||
zap.String("failed reason", status.Reason))
|
||||
return fmt.Errorf(status.Reason)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (broker *globalMetaBroker) releaseSegmentReferLock(ctx context.Context, segmentIDs []UniqueID) error {
|
||||
ctx, cancel := context.WithTimeout(ctx, timeoutForRPC)
|
||||
defer cancel()
|
||||
|
||||
releaseSegReferLockReq := &datapb.ReleaseSegmentLockRequest{
|
||||
SegmentIDs: segmentIDs,
|
||||
NodeID: Params.QueryCoordCfg.GetNodeID(),
|
||||
}
|
||||
|
||||
if err := retry.Do(ctx, func() error {
|
||||
status, err := broker.dataCoord.ReleaseSegmentLock(ctx, releaseSegReferLockReq)
|
||||
if err != nil {
|
||||
log.Error("QueryCoord release reference lock on segments failed", zap.Int64s("segmentIDs", segmentIDs),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
log.Error("QueryCoord release reference lock on segments failed", zap.Int64s("segmentIDs", segmentIDs),
|
||||
zap.String("failed reason", status.Reason))
|
||||
return errors.New(status.Reason)
|
||||
}
|
||||
return nil
|
||||
}, retry.Attempts(100)); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -334,6 +334,7 @@ type dataCoordMock struct {
|
||||
returnError bool
|
||||
returnGrpcError bool
|
||||
segmentState commonpb.SegmentState
|
||||
errLevel int
|
||||
}
|
||||
|
||||
func newDataCoordMock(ctx context.Context) *dataCoordMock {
|
||||
@ -448,6 +449,42 @@ func (data *dataCoordMock) GetSegmentStates(ctx context.Context, req *datapb.Get
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (data *dataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
|
||||
if data.errLevel == 2 {
|
||||
data.errLevel++
|
||||
return nil, errors.New("AcquireSegmentLock failed")
|
||||
|
||||
}
|
||||
if data.errLevel == 1 {
|
||||
data.errLevel++
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "AcquireSegmentLock failed",
|
||||
}, nil
|
||||
}
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
}, nil
|
||||
}
|
||||
func (data *dataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
|
||||
if data.errLevel == 4 {
|
||||
data.errLevel++
|
||||
return nil, errors.New("ReleaseSegmentLock failed")
|
||||
}
|
||||
|
||||
if data.errLevel == 3 {
|
||||
data.errLevel++
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "ReleaseSegmentLock failed",
|
||||
}, nil
|
||||
}
|
||||
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
}, nil
|
||||
}
|
||||
|
||||
type indexCoordMock struct {
|
||||
types.IndexCoord
|
||||
chunkManager storage.ChunkManager
|
||||
|
||||
@ -509,7 +509,7 @@ func (lct *loadCollectionTask) execute(ctx context.Context) error {
|
||||
watchDmChannelReqs = append(watchDmChannelReqs, watchRequest)
|
||||
}
|
||||
|
||||
internalTasks, err := assignInternalTask(ctx, lct, lct.meta, lct.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1)
|
||||
internalTasks, err := assignInternalTask(ctx, lct, lct.meta, lct.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1, lct.broker)
|
||||
if err != nil {
|
||||
log.Error("loadCollectionTask: assign child task failed", zap.Int64("collectionID", collectionID), zap.Int64("msgID", lct.Base.MsgID), zap.Error(err))
|
||||
lct.setResultInfo(err)
|
||||
@ -939,7 +939,7 @@ func (lpt *loadPartitionTask) execute(ctx context.Context) error {
|
||||
watchDmChannelReqs = append(watchDmChannelReqs, watchRequest)
|
||||
}
|
||||
|
||||
internalTasks, err := assignInternalTask(ctx, lpt, lpt.meta, lpt.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1)
|
||||
internalTasks, err := assignInternalTask(ctx, lpt, lpt.meta, lpt.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1, lpt.broker)
|
||||
if err != nil {
|
||||
log.Error("loadPartitionTask: assign child task failed", zap.Int64("collectionID", collectionID), zap.Int64s("partitionIDs", partitionIDs), zap.Int64("msgID", lpt.Base.MsgID), zap.Error(err))
|
||||
lpt.setResultInfo(err)
|
||||
@ -1176,6 +1176,8 @@ type loadSegmentTask struct {
|
||||
meta Meta
|
||||
cluster Cluster
|
||||
excludeNodeIDs []int64
|
||||
|
||||
broker *globalMetaBroker
|
||||
}
|
||||
|
||||
func (lst *loadSegmentTask) msgBase() *commonpb.MsgBase {
|
||||
@ -1212,7 +1214,7 @@ func (lst *loadSegmentTask) updateTaskProcess() {
|
||||
parentTask.updateTaskProcess()
|
||||
}
|
||||
|
||||
func (lst *loadSegmentTask) preExecute(context.Context) error {
|
||||
func (lst *loadSegmentTask) preExecute(ctx context.Context) error {
|
||||
segmentIDs := make([]UniqueID, 0)
|
||||
for _, info := range lst.Infos {
|
||||
segmentIDs = append(segmentIDs, info.SegmentID)
|
||||
@ -1222,6 +1224,12 @@ func (lst *loadSegmentTask) preExecute(context.Context) error {
|
||||
zap.Int64s("segmentIDs", segmentIDs),
|
||||
zap.Int64("loaded nodeID", lst.DstNodeID),
|
||||
zap.Int64("taskID", lst.getTaskID()))
|
||||
|
||||
if err := lst.broker.acquireSegmentsReferLock(ctx, segmentIDs); err != nil {
|
||||
log.Error("acquire reference lock on segments failed", zap.Int64s("segmentIDs", segmentIDs),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -1241,6 +1249,14 @@ func (lst *loadSegmentTask) execute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
func (lst *loadSegmentTask) postExecute(context.Context) error {
|
||||
segmentIDs := make([]UniqueID, 0)
|
||||
for _, info := range lst.Infos {
|
||||
segmentIDs = append(segmentIDs, info.SegmentID)
|
||||
}
|
||||
if err := lst.broker.releaseSegmentReferLock(lst.ctx, segmentIDs); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
log.Info("loadSegmentTask postExecute done",
|
||||
zap.Int64("taskID", lst.getTaskID()))
|
||||
return nil
|
||||
@ -1275,7 +1291,7 @@ func (lst *loadSegmentTask) reschedule(ctx context.Context) ([]task, error) {
|
||||
if lst.getParentTask().getTriggerCondition() == querypb.TriggerCondition_NodeDown {
|
||||
wait2AssignTaskSuccess = true
|
||||
}
|
||||
reScheduledTasks, err := assignInternalTask(ctx, lst.getParentTask(), lst.meta, lst.cluster, loadSegmentReqs, nil, wait2AssignTaskSuccess, lst.excludeNodeIDs, nil, lst.ReplicaID)
|
||||
reScheduledTasks, err := assignInternalTask(ctx, lst.getParentTask(), lst.meta, lst.cluster, loadSegmentReqs, nil, wait2AssignTaskSuccess, lst.excludeNodeIDs, nil, lst.ReplicaID, lst.broker)
|
||||
if err != nil {
|
||||
log.Error("loadSegment reschedule failed", zap.Int64s("excludeNodes", lst.excludeNodeIDs), zap.Int64("taskID", lst.getTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
@ -1456,7 +1472,7 @@ func (wdt *watchDmChannelTask) reschedule(ctx context.Context) ([]task, error) {
|
||||
if wdt.getParentTask().getTriggerCondition() == querypb.TriggerCondition_NodeDown {
|
||||
wait2AssignTaskSuccess = true
|
||||
}
|
||||
reScheduledTasks, err := assignInternalTask(ctx, wdt.parentTask, wdt.meta, wdt.cluster, nil, watchDmChannelReqs, wait2AssignTaskSuccess, wdt.excludeNodeIDs, nil, wdt.ReplicaID)
|
||||
reScheduledTasks, err := assignInternalTask(ctx, wdt.parentTask, wdt.meta, wdt.cluster, nil, watchDmChannelReqs, wait2AssignTaskSuccess, wdt.excludeNodeIDs, nil, wdt.ReplicaID, nil)
|
||||
if err != nil {
|
||||
log.Error("watchDmChannel reschedule failed", zap.Int64("taskID", wdt.getTaskID()), zap.Int64s("excludeNodes", wdt.excludeNodeIDs), zap.Error(err))
|
||||
return nil, err
|
||||
@ -1591,7 +1607,7 @@ func (ht *handoffTask) execute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
if collectionInfo.LoadType == querypb.LoadType_LoadCollection && ht.meta.hasReleasePartition(collectionID, partitionID) {
|
||||
log.Warn("handoffTask: partition has not been released", zap.Int64("collectionID", collectionID), zap.Int64("partitionID", partitionID))
|
||||
log.Warn("handoffTask: partition has been released", zap.Int64("collectionID", collectionID), zap.Int64("partitionID", partitionID))
|
||||
continue
|
||||
}
|
||||
|
||||
@ -1687,7 +1703,7 @@ func (ht *handoffTask) execute(ctx context.Context) error {
|
||||
// we should copy a request because assignInternalTask will change DstNodeID of LoadSegmentRequest
|
||||
clonedReq := proto.Clone(loadSegmentReq).(*querypb.LoadSegmentsRequest)
|
||||
clonedReq.ReplicaID = replica.ReplicaID
|
||||
tasks, err := assignInternalTask(ctx, ht, ht.meta, ht.cluster, []*querypb.LoadSegmentsRequest{clonedReq}, nil, true, nil, nil, replica.GetReplicaID())
|
||||
tasks, err := assignInternalTask(ctx, ht, ht.meta, ht.cluster, []*querypb.LoadSegmentsRequest{clonedReq}, nil, true, nil, nil, replica.GetReplicaID(), ht.broker)
|
||||
if err != nil {
|
||||
log.Error("handoffTask: assign child task failed", zap.Int64("collectionID", collectionID), zap.Int64("segmentID", segmentID), zap.Error(err))
|
||||
ht.setResultInfo(err)
|
||||
@ -1960,7 +1976,7 @@ func (lbt *loadBalanceTask) execute(ctx context.Context) error {
|
||||
}
|
||||
}
|
||||
|
||||
tasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, watchDmChannelReqs, true, lbt.SourceNodeIDs, lbt.DstNodeIDs, replica.GetReplicaID())
|
||||
tasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, watchDmChannelReqs, true, lbt.SourceNodeIDs, lbt.DstNodeIDs, replica.GetReplicaID(), lbt.broker)
|
||||
if err != nil {
|
||||
log.Error("loadBalanceTask: assign child task failed", zap.Int64("sourceNodeID", nodeID))
|
||||
lbt.setResultInfo(err)
|
||||
@ -2144,7 +2160,7 @@ func (lbt *loadBalanceTask) execute(ctx context.Context) error {
|
||||
return err
|
||||
}
|
||||
}
|
||||
internalTasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, nil, false, lbt.SourceNodeIDs, lbt.DstNodeIDs, lbt.replicaID)
|
||||
internalTasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, nil, false, lbt.SourceNodeIDs, lbt.DstNodeIDs, lbt.replicaID, lbt.broker)
|
||||
if err != nil {
|
||||
log.Error("loadBalanceTask: assign child task failed", zap.Any("balance request", lbt.LoadBalanceRequest))
|
||||
lbt.setResultInfo(err)
|
||||
@ -2350,7 +2366,8 @@ func assignInternalTask(ctx context.Context,
|
||||
parentTask task, meta Meta, cluster Cluster,
|
||||
loadSegmentRequests []*querypb.LoadSegmentsRequest,
|
||||
watchDmChannelRequests []*querypb.WatchDmChannelsRequest,
|
||||
wait bool, excludeNodeIDs []int64, includeNodeIDs []int64, replicaID int64) ([]task, error) {
|
||||
wait bool, excludeNodeIDs []int64, includeNodeIDs []int64, replicaID int64,
|
||||
broker *globalMetaBroker) ([]task, error) {
|
||||
|
||||
internalTasks := make([]task, 0)
|
||||
err := cluster.allocateSegmentsToQueryNode(ctx, loadSegmentRequests, wait, excludeNodeIDs, includeNodeIDs, replicaID)
|
||||
@ -2387,6 +2404,7 @@ func assignInternalTask(ctx context.Context,
|
||||
meta: meta,
|
||||
cluster: cluster,
|
||||
excludeNodeIDs: excludeNodeIDs,
|
||||
broker: broker,
|
||||
}
|
||||
internalTasks = append(internalTasks, loadSegmentTask)
|
||||
|
||||
@ -2407,6 +2425,7 @@ func assignInternalTask(ctx context.Context,
|
||||
meta: meta,
|
||||
cluster: cluster,
|
||||
excludeNodeIDs: excludeNodeIDs,
|
||||
broker: broker,
|
||||
}
|
||||
internalTasks = append(internalTasks, loadSegmentTask)
|
||||
}
|
||||
|
||||
@ -336,6 +336,7 @@ func (scheduler *TaskScheduler) unmarshalTask(taskID UniqueID, t string) (task,
|
||||
cluster: scheduler.cluster,
|
||||
meta: scheduler.meta,
|
||||
excludeNodeIDs: []int64{},
|
||||
broker: scheduler.broker,
|
||||
}
|
||||
newTask = loadSegmentTask
|
||||
case commonpb.MsgType_ReleaseSegments:
|
||||
@ -521,7 +522,11 @@ func (scheduler *TaskScheduler) processTask(t task) error {
|
||||
|
||||
// task preExecute
|
||||
span.LogFields(oplog.Int64("processTask: scheduler process PreExecute", t.getTaskID()))
|
||||
t.preExecute(ctx)
|
||||
err = t.preExecute(ctx)
|
||||
if err != nil {
|
||||
log.Warn("failed to preExecute task", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
taskInfoKey = fmt.Sprintf("%s/%d", taskInfoPrefix, t.getTaskID())
|
||||
err = scheduler.client.Save(taskInfoKey, strconv.Itoa(int(taskDoing)))
|
||||
if err != nil {
|
||||
|
||||
@ -18,9 +18,13 @@ package querycoord
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/zap"
|
||||
|
||||
@ -213,13 +217,14 @@ func genLoadSegmentTask(ctx context.Context, queryCoord *QueryCoord, nodeID int6
|
||||
},
|
||||
}
|
||||
baseTask := newBaseTask(ctx, querypb.TriggerCondition_GrpcRequest)
|
||||
baseTask.taskID = 100
|
||||
baseTask.taskID = rand.Int63()
|
||||
loadSegmentTask := &loadSegmentTask{
|
||||
baseTask: baseTask,
|
||||
LoadSegmentsRequest: req,
|
||||
cluster: queryCoord.cluster,
|
||||
meta: queryCoord.meta,
|
||||
excludeNodeIDs: []int64{},
|
||||
broker: queryCoord.broker,
|
||||
}
|
||||
|
||||
parentReq := &querypb.LoadCollectionRequest{
|
||||
@ -764,7 +769,7 @@ func Test_AssignInternalTask(t *testing.T) {
|
||||
loadSegmentRequests = append(loadSegmentRequests, req)
|
||||
}
|
||||
|
||||
internalTasks, err := assignInternalTask(queryCoord.loopCtx, loadCollectionTask, queryCoord.meta, queryCoord.cluster, loadSegmentRequests, nil, false, nil, nil, -1)
|
||||
internalTasks, err := assignInternalTask(queryCoord.loopCtx, loadCollectionTask, queryCoord.meta, queryCoord.cluster, loadSegmentRequests, nil, false, nil, nil, -1, nil)
|
||||
assert.Nil(t, err)
|
||||
|
||||
assert.NotEqual(t, 1, len(internalTasks))
|
||||
@ -1377,3 +1382,82 @@ func TestUpdateTaskProcessWhenWatchDmChannel(t *testing.T) {
|
||||
err = removeAllSession()
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func startMockCoord(ctx context.Context) (*QueryCoord, error) {
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
coord, err := NewQueryCoordTest(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rootCoord := newRootCoordMock(ctx)
|
||||
rootCoord.createCollection(defaultCollectionID)
|
||||
rootCoord.createPartition(defaultCollectionID, defaultPartitionID)
|
||||
|
||||
dataCoord := &dataCoordMock{
|
||||
collections: make([]UniqueID, 0),
|
||||
col2DmChannels: make(map[UniqueID][]*datapb.VchannelInfo),
|
||||
partitionID2Segment: make(map[UniqueID][]UniqueID),
|
||||
Segment2Binlog: make(map[UniqueID]*datapb.SegmentBinlogs),
|
||||
baseSegmentID: defaultSegmentID,
|
||||
channelNumPerCol: defaultChannelNum,
|
||||
segmentState: commonpb.SegmentState_Flushed,
|
||||
errLevel: 1,
|
||||
}
|
||||
indexCoord, err := newIndexCoordMock(queryCoordTestDir)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
coord.SetRootCoord(rootCoord)
|
||||
coord.SetDataCoord(dataCoord)
|
||||
coord.SetIndexCoord(indexCoord)
|
||||
etcd, err := etcd.GetEtcdClient(&Params.EtcdCfg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coord.SetEtcdClient(etcd)
|
||||
err = coord.Init()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = coord.Start()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = coord.Register()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return coord, nil
|
||||
}
|
||||
|
||||
func Test_LoadSegment(t *testing.T) {
|
||||
refreshParams()
|
||||
ctx := context.Background()
|
||||
queryCoord, err := startMockCoord(ctx)
|
||||
assert.Nil(t, err)
|
||||
|
||||
node1, err := startQueryNodeServer(ctx)
|
||||
assert.Nil(t, err)
|
||||
|
||||
waitQueryNodeOnline(queryCoord.cluster, node1.queryNodeID)
|
||||
|
||||
loadSegmentTask := genLoadSegmentTask(ctx, queryCoord, node1.queryNodeID)
|
||||
err = loadSegmentTask.meta.setDeltaChannel(111, []*datapb.VchannelInfo{})
|
||||
assert.Nil(t, err)
|
||||
loadCollectionTask := loadSegmentTask.parentTask
|
||||
queryCoord.scheduler.triggerTaskQueue.addTask(loadCollectionTask)
|
||||
|
||||
// 1. Acquire segment reference lock failed, and reschedule task.
|
||||
// 2. Acquire segment reference lock successfully, but release reference lock failed, and retry release the lock.
|
||||
// 3. Release segment reference lock successfully, and task done.
|
||||
waitTaskFinalState(loadSegmentTask, taskDone)
|
||||
|
||||
err = queryCoord.Stop()
|
||||
assert.Nil(t, err)
|
||||
|
||||
err = removeAllSession()
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
@ -128,13 +128,12 @@ type Core struct {
|
||||
//setMsgStreams, send drop partition into dd channel
|
||||
SendDdDropPartitionReq func(ctx context.Context, req *internalpb.DropPartitionRequest, channelNames []string) error
|
||||
|
||||
//get binlog file path from data service,
|
||||
CallGetBinlogFilePathsService func(ctx context.Context, segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error)
|
||||
CallGetNumRowsService func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error)
|
||||
//get segment info from data service
|
||||
CallGetFlushedSegmentsService func(ctx context.Context, collID, partID typeutil.UniqueID) ([]typeutil.UniqueID, error)
|
||||
CallGetRecoveryInfoService func(ctx context.Context, collID, partID UniqueID) ([]*datapb.SegmentBinlogs, error)
|
||||
|
||||
//call index builder's client to build index, return build id or get index state.
|
||||
CallBuildIndexService func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error)
|
||||
CallBuildIndexService func(ctx context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error)
|
||||
CallDropIndexService func(ctx context.Context, indexID typeutil.UniqueID) error
|
||||
CallGetIndexStatesService func(ctx context.Context, IndexBuildIDs []int64) ([]*indexpb.IndexInfo, error)
|
||||
|
||||
@ -264,11 +263,11 @@ func (c *Core) checkInit() error {
|
||||
if c.SendDdDropPartitionReq == nil {
|
||||
return fmt.Errorf("sendDdDropPartitionReq is nil")
|
||||
}
|
||||
if c.CallGetBinlogFilePathsService == nil {
|
||||
return fmt.Errorf("callGetBinlogFilePathsService is nil")
|
||||
if c.CallGetFlushedSegmentsService == nil {
|
||||
return fmt.Errorf("callGetFlushedSegmentsService is nil")
|
||||
}
|
||||
if c.CallGetNumRowsService == nil {
|
||||
return fmt.Errorf("callGetNumRowsService is nil")
|
||||
if c.CallGetRecoveryInfoService == nil {
|
||||
return fmt.Errorf("CallGetRecoveryInfoService is nil")
|
||||
}
|
||||
if c.CallBuildIndexService == nil {
|
||||
return fmt.Errorf("callBuildIndexService is nil")
|
||||
@ -276,9 +275,6 @@ func (c *Core) checkInit() error {
|
||||
if c.CallDropIndexService == nil {
|
||||
return fmt.Errorf("callDropIndexService is nil")
|
||||
}
|
||||
if c.CallGetFlushedSegmentsService == nil {
|
||||
return fmt.Errorf("callGetFlushedSegmentsService is nil")
|
||||
}
|
||||
if c.CallWatchChannels == nil {
|
||||
return fmt.Errorf("callWatchChannels is nil")
|
||||
}
|
||||
@ -369,7 +365,7 @@ func (c *Core) checkFlushedSegments(ctx context.Context) {
|
||||
}
|
||||
for _, partID := range collMeta.PartitionIDs {
|
||||
ctx2, cancel2 := context.WithTimeout(ctx, 3*time.Minute)
|
||||
segIDs, err := c.CallGetFlushedSegmentsService(ctx2, collMeta.ID, partID)
|
||||
segBinlogs, err := c.CallGetRecoveryInfoService(ctx2, collMeta.ID, partID)
|
||||
if err != nil {
|
||||
log.Debug("failed to get flushed segments from dataCoord",
|
||||
zap.Int64("collection ID", collMeta.GetID()),
|
||||
@ -378,7 +374,8 @@ func (c *Core) checkFlushedSegments(ctx context.Context) {
|
||||
cancel2()
|
||||
continue
|
||||
}
|
||||
for _, segID := range segIDs {
|
||||
for _, segBinlog := range segBinlogs {
|
||||
segID := segBinlog.SegmentID
|
||||
var indexInfos []*etcdpb.FieldIndexInfo
|
||||
indexMeta, ok := segID2IndexMeta[segID]
|
||||
if !ok {
|
||||
@ -416,7 +413,7 @@ func (c *Core) checkFlushedSegments(ctx context.Context) {
|
||||
zap.Int64("segment_id", segID),
|
||||
zap.Int64("index_id", indexMeta.IndexID),
|
||||
zap.Int64("collection_id", collMeta.ID))
|
||||
info.BuildID, err = c.BuildIndex(ctx2, segID, field, &indexMeta, false)
|
||||
info.BuildID, err = c.BuildIndex(ctx2, segID, segBinlog.GetNumOfRows(), segBinlog.GetFieldBinlogs(), field, &indexMeta, false)
|
||||
if err != nil {
|
||||
log.Debug("build index failed",
|
||||
zap.Int64("segment_id", segID),
|
||||
@ -441,27 +438,29 @@ func (c *Core) checkFlushedSegments(ctx context.Context) {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Core) getSegments(ctx context.Context, collID typeutil.UniqueID) (map[typeutil.UniqueID]typeutil.UniqueID, error) {
|
||||
func (c *Core) getSegments(ctx context.Context, collID typeutil.UniqueID) (map[UniqueID]UniqueID, map[UniqueID]*datapb.SegmentBinlogs, error) {
|
||||
collMeta, err := c.MetaTable.GetCollectionByID(collID, 0)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
segID2PartID := map[typeutil.UniqueID]typeutil.UniqueID{}
|
||||
segID2PartID := make(map[UniqueID]UniqueID)
|
||||
segID2Binlog := make(map[UniqueID]*datapb.SegmentBinlogs)
|
||||
for _, partID := range collMeta.PartitionIDs {
|
||||
if seg, err := c.CallGetFlushedSegmentsService(ctx, collID, partID); err == nil {
|
||||
for _, s := range seg {
|
||||
segID2PartID[s] = partID
|
||||
if segs, err := c.CallGetRecoveryInfoService(ctx, collID, partID); err == nil {
|
||||
for _, s := range segs {
|
||||
segID2PartID[s.SegmentID] = partID
|
||||
segID2Binlog[s.SegmentID] = s
|
||||
}
|
||||
} else {
|
||||
log.Error("failed to get flushed segments from dataCoord",
|
||||
log.Error("failed to get flushed segments info from dataCoord",
|
||||
zap.Int64("collection ID", collID),
|
||||
zap.Int64("partition ID", partID),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
return nil, nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return segID2PartID, nil
|
||||
return segID2PartID, segID2Binlog, nil
|
||||
}
|
||||
|
||||
func (c *Core) setMsgStreams() error {
|
||||
@ -580,80 +579,6 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
|
||||
log.Debug("Retrying RootCoord connection to DataCoord")
|
||||
}
|
||||
}()
|
||||
c.CallGetBinlogFilePathsService = func(ctx context.Context, segID typeutil.UniqueID, fieldID typeutil.UniqueID) (retFiles []string, retErr error) {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
retErr = fmt.Errorf("get bin log file paths panic, msg = %v", err)
|
||||
}
|
||||
}()
|
||||
<-initCh //wait connect to DataCoord
|
||||
ts, err := c.TSOAllocator(1)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
binlog, err := s.GetInsertBinlogPaths(ctx, &datapb.GetInsertBinlogPathsRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: 0, //TODO, msg type
|
||||
MsgID: 0,
|
||||
Timestamp: ts,
|
||||
SourceID: c.session.ServerID,
|
||||
},
|
||||
SegmentID: segID,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if binlog.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
return nil, fmt.Errorf("getInsertBinlogPaths from data service failed, error = %s", binlog.Status.Reason)
|
||||
}
|
||||
binlogPaths := make([]string, 0)
|
||||
for i := range binlog.FieldIDs {
|
||||
if binlog.FieldIDs[i] == fieldID {
|
||||
binlogPaths = append(binlogPaths, binlog.Paths[i].Values...)
|
||||
}
|
||||
}
|
||||
if len(binlogPaths) == 0 {
|
||||
return nil, fmt.Errorf("binlog file does not exist, segment id = %d, field id = %d", segID, fieldID)
|
||||
}
|
||||
return binlogPaths, nil
|
||||
}
|
||||
|
||||
c.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (retRows int64, retErr error) {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
retErr = fmt.Errorf("get num rows panic, msg = %v", err)
|
||||
}
|
||||
}()
|
||||
<-initCh
|
||||
ts, err := c.TSOAllocator(1)
|
||||
if err != nil {
|
||||
return retRows, err
|
||||
}
|
||||
segInfo, err := s.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: 0, //TODO, msg type
|
||||
MsgID: 0,
|
||||
Timestamp: ts,
|
||||
SourceID: c.session.ServerID,
|
||||
},
|
||||
SegmentIDs: []typeutil.UniqueID{segID},
|
||||
})
|
||||
if err != nil {
|
||||
return retRows, err
|
||||
}
|
||||
if segInfo.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
return retRows, fmt.Errorf("getSegmentInfo from data service failed, error = %s", segInfo.Status.Reason)
|
||||
}
|
||||
if len(segInfo.Infos) != 1 {
|
||||
log.Debug("get segment info empty")
|
||||
return retRows, nil
|
||||
}
|
||||
if !isFromFlushedChan && segInfo.Infos[0].State != commonpb.SegmentState_Flushed {
|
||||
log.Debug("segment id not flushed", zap.Int64("segment id", segID))
|
||||
return retRows, nil
|
||||
}
|
||||
return segInfo.Infos[0].NumOfRows, nil
|
||||
}
|
||||
|
||||
c.CallGetFlushedSegmentsService = func(ctx context.Context, collID, partID typeutil.UniqueID) (retSegIDs []typeutil.UniqueID, retErr error) {
|
||||
defer func() {
|
||||
@ -682,6 +607,27 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
|
||||
return rsp.Segments, nil
|
||||
}
|
||||
|
||||
c.CallGetRecoveryInfoService = func(ctx context.Context, collID, partID typeutil.UniqueID) ([]*datapb.SegmentBinlogs, error) {
|
||||
getSegmentInfoReq := &datapb.GetRecoveryInfoRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: 0, //TODO, msg type
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: c.session.ServerID,
|
||||
},
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
}
|
||||
resp, err := s.GetRecoveryInfo(ctx, getSegmentInfoReq)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
return nil, errors.New(resp.Status.Reason)
|
||||
}
|
||||
return resp.Binlogs, nil
|
||||
}
|
||||
|
||||
c.CallWatchChannels = func(ctx context.Context, collectionID int64, channelNames []string) (retErr error) {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
@ -758,7 +704,7 @@ func (c *Core) SetIndexCoord(s types.IndexCoord) error {
|
||||
}
|
||||
}()
|
||||
|
||||
c.CallBuildIndexService = func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (retID typeutil.UniqueID, retErr error) {
|
||||
c.CallBuildIndexService = func(ctx context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (retID typeutil.UniqueID, retErr error) {
|
||||
defer func() {
|
||||
if err := recover(); err != nil {
|
||||
retErr = fmt.Errorf("build index panic, msg = %v", err)
|
||||
@ -773,6 +719,7 @@ func (c *Core) SetIndexCoord(s types.IndexCoord) error {
|
||||
IndexName: idxInfo.IndexName,
|
||||
NumRows: numRows,
|
||||
FieldSchema: field,
|
||||
SegmentID: segID,
|
||||
})
|
||||
if err != nil {
|
||||
return retID, err
|
||||
@ -917,7 +864,7 @@ func (c *Core) SetQueryCoord(s types.QueryCoord) error {
|
||||
}
|
||||
|
||||
// BuildIndex will check row num and call build index service
|
||||
func (c *Core) BuildIndex(ctx context.Context, segID typeutil.UniqueID, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, isFlush bool) (typeutil.UniqueID, error) {
|
||||
func (c *Core) BuildIndex(ctx context.Context, segID UniqueID, numRows int64, binlogs []*datapb.FieldBinlog, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, isFlush bool) (typeutil.UniqueID, error) {
|
||||
log.Debug("start build index", zap.String("index name", idxInfo.IndexName),
|
||||
zap.String("field name", field.Name), zap.Int64("segment id", segID))
|
||||
sp, ctx := trace.StartSpanFromContext(ctx)
|
||||
@ -926,27 +873,24 @@ func (c *Core) BuildIndex(ctx context.Context, segID typeutil.UniqueID, field *s
|
||||
info, err := c.MetaTable.GetSegmentIndexInfoByID(segID, field.FieldID, idxInfo.GetIndexName())
|
||||
return info.BuildID, err
|
||||
}
|
||||
rows, err := c.CallGetNumRowsService(ctx, segID, isFlush)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
var bldID typeutil.UniqueID
|
||||
if rows < Params.RootCoordCfg.MinSegmentSizeToEnableIndex {
|
||||
log.Debug("num of rows is less than MinSegmentSizeToEnableIndex", zap.Int64("num rows", rows))
|
||||
var bldID UniqueID
|
||||
var err error
|
||||
if numRows < Params.RootCoordCfg.MinSegmentSizeToEnableIndex {
|
||||
log.Debug("num of rows is less than MinSegmentSizeToEnableIndex", zap.Int64("num rows", numRows))
|
||||
} else {
|
||||
binlogs, err := c.CallGetBinlogFilePathsService(ctx, segID, field.FieldID)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
binLogs := make([]string, 0)
|
||||
for _, fieldBinLog := range binlogs {
|
||||
if fieldBinLog.GetFieldID() == field.GetFieldID() {
|
||||
for _, binLog := range fieldBinLog.GetBinlogs() {
|
||||
binLogs = append(binLogs, binLog.LogPath)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
bldID, err = c.CallBuildIndexService(ctx, binlogs, field, idxInfo, rows)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
log.Debug("CallBuildIndex finished", zap.String("index name", idxInfo.IndexName),
|
||||
zap.String("field name", field.Name), zap.Int64("segment id", segID), zap.Int64("num rows", rows))
|
||||
bldID, err = c.CallBuildIndexService(ctx, segID, binLogs, field, idxInfo, numRows)
|
||||
}
|
||||
return bldID, nil
|
||||
|
||||
return bldID, err
|
||||
}
|
||||
|
||||
// RemoveIndex will call drop index service
|
||||
@ -2136,7 +2080,7 @@ func (c *Core) SegmentFlushCompleted(ctx context.Context, in *datapb.SegmentFlus
|
||||
IndexID: idxInfo.IndexID,
|
||||
EnableIndex: false,
|
||||
}
|
||||
info.BuildID, err = c.BuildIndex(ctx, segID, fieldSch, idxInfo, true)
|
||||
info.BuildID, err = c.BuildIndex(ctx, segID, in.Segment.GetNumOfRows(), in.Segment.GetBinlogs(), fieldSch, idxInfo, true)
|
||||
if err == nil && info.BuildID != 0 {
|
||||
info.EnableIndex = true
|
||||
} else {
|
||||
|
||||
@ -199,6 +199,47 @@ func (d *dataMock) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInf
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (d *dataMock) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
|
||||
var fieldBinlog []*datapb.FieldBinlog
|
||||
for i := 0; i < 200; i++ {
|
||||
binlog := &datapb.FieldBinlog{
|
||||
FieldID: int64(i),
|
||||
Binlogs: []*datapb.Binlog{
|
||||
{
|
||||
LogPath: fmt.Sprintf("file0-%d", i),
|
||||
},
|
||||
{
|
||||
LogPath: fmt.Sprintf("file1-%d", i),
|
||||
},
|
||||
{
|
||||
LogPath: fmt.Sprintf("file2-%d", i),
|
||||
},
|
||||
},
|
||||
}
|
||||
fieldBinlog = append(fieldBinlog, binlog)
|
||||
}
|
||||
|
||||
d.mu.Lock()
|
||||
segmentBinlogs := make([]*datapb.SegmentBinlogs, 0, len(d.segs))
|
||||
for _, segID := range d.segs {
|
||||
segmentBinlog := &datapb.SegmentBinlogs{
|
||||
SegmentID: segID,
|
||||
NumOfRows: Params.RootCoordCfg.MinSegmentSizeToEnableIndex,
|
||||
FieldBinlogs: fieldBinlog,
|
||||
}
|
||||
segmentBinlogs = append(segmentBinlogs, segmentBinlog)
|
||||
}
|
||||
d.mu.Unlock()
|
||||
|
||||
return &datapb.GetRecoveryInfoResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
Reason: "",
|
||||
},
|
||||
Binlogs: segmentBinlogs,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (d *dataMock) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
|
||||
d.mu.Lock()
|
||||
defer d.mu.Unlock()
|
||||
@ -2939,25 +2980,21 @@ func TestCheckInit(t *testing.T) {
|
||||
err = c.checkInit()
|
||||
assert.Error(t, err)
|
||||
|
||||
c.CallGetBinlogFilePathsService = func(ctx context.Context, segID, fieldID typeutil.UniqueID) ([]string, error) {
|
||||
return []string{}, nil
|
||||
}
|
||||
err = c.checkInit()
|
||||
assert.Error(t, err)
|
||||
|
||||
c.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error) {
|
||||
return 0, nil
|
||||
}
|
||||
err = c.checkInit()
|
||||
assert.Error(t, err)
|
||||
|
||||
c.CallGetFlushedSegmentsService = func(ctx context.Context, collID, partID typeutil.UniqueID) ([]typeutil.UniqueID, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
err = c.checkInit()
|
||||
assert.Error(t, err)
|
||||
|
||||
c.CallBuildIndexService = func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) {
|
||||
c.CallGetRecoveryInfoService = func(ctx context.Context, collID, partID UniqueID) ([]*datapb.SegmentBinlogs, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
err = c.checkInit()
|
||||
assert.Error(t, err)
|
||||
|
||||
c.CallBuildIndexService = func(ctx context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) {
|
||||
return 0, nil
|
||||
}
|
||||
err = c.checkInit()
|
||||
@ -3142,7 +3179,7 @@ func TestCheckFlushedSegments(t *testing.T) {
|
||||
core.MetaTable.indexID2Meta[indexID] = etcdpb.IndexInfo{
|
||||
IndexID: indexID,
|
||||
}
|
||||
core.CallBuildIndexService = func(_ context.Context, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) {
|
||||
core.CallBuildIndexService = func(_ context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) {
|
||||
assert.Equal(t, fieldID, field.FieldID)
|
||||
assert.Equal(t, indexID, idx.IndexID)
|
||||
return -1, errors.New("build index build")
|
||||
@ -3151,7 +3188,7 @@ func TestCheckFlushedSegments(t *testing.T) {
|
||||
core.checkFlushedSegments(ctx)
|
||||
|
||||
var indexBuildID int64 = 10001
|
||||
core.CallBuildIndexService = func(_ context.Context, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) {
|
||||
core.CallBuildIndexService = func(_ context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) {
|
||||
return indexBuildID, nil
|
||||
}
|
||||
core.checkFlushedSegments(core.ctx)
|
||||
|
||||
@ -959,7 +959,7 @@ func (t *CreateIndexReqTask) Execute(ctx context.Context) error {
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
segID2PartID, err := t.core.getSegments(ctx, collMeta.ID)
|
||||
segID2PartID, segID2Binlog, err := t.core.getSegments(ctx, collMeta.ID)
|
||||
flushedSegs := make([]typeutil.UniqueID, 0, len(segID2PartID))
|
||||
for k := range segID2PartID {
|
||||
flushedSegs = append(flushedSegs, k)
|
||||
@ -987,7 +987,7 @@ func (t *CreateIndexReqTask) Execute(ctx context.Context) error {
|
||||
IndexID: idxInfo.IndexID,
|
||||
EnableIndex: false,
|
||||
}
|
||||
info.BuildID, err = t.core.BuildIndex(ctx, segID, &field, idxInfo, false)
|
||||
info.BuildID, err = t.core.BuildIndex(ctx, segID, segID2Binlog[segID].GetNumOfRows(), segID2Binlog[segID].GetFieldBinlogs(), &field, idxInfo, false)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@ -281,6 +281,9 @@ type DataCoord interface {
|
||||
|
||||
// UpdateSegmentStatistics updates a segment's stats.
|
||||
UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error)
|
||||
|
||||
AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error)
|
||||
ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error)
|
||||
}
|
||||
|
||||
// DataCoordComponent defines the interface of DataCoord component.
|
||||
@ -350,6 +353,8 @@ type IndexCoordComponent interface {
|
||||
// SetEtcdClient set etcd client for IndexCoordComponent
|
||||
SetEtcdClient(etcdClient *clientv3.Client)
|
||||
|
||||
SetDataCoord(dataCoor DataCoord) error
|
||||
|
||||
// UpdateStateCode updates state code for IndexCoordComponent
|
||||
// `stateCode` is current statement of this IndexCoordComponent, indicating whether it's healthy.
|
||||
UpdateStateCode(stateCode internalpb.StateCode)
|
||||
|
||||
140
internal/util/mock/datacoord_client.go
Normal file
140
internal/util/mock/datacoord_client.go
Normal file
@ -0,0 +1,140 @@
|
||||
// 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 mock
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
)
|
||||
|
||||
// DataCoordClient mocks of DataCoordClient
|
||||
type DataCoordClient struct {
|
||||
Err error
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetComponentStates(ctx context.Context, in *internalpb.GetComponentStatesRequest, opts ...grpc.CallOption) (*internalpb.ComponentStates, error) {
|
||||
return &internalpb.ComponentStates{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
return &milvuspb.StringResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
return &milvuspb.StringResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) Flush(ctx context.Context, in *datapb.FlushRequest, opts ...grpc.CallOption) (*datapb.FlushResponse, error) {
|
||||
return &datapb.FlushResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) AssignSegmentID(ctx context.Context, in *datapb.AssignSegmentIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegmentIDResponse, error) {
|
||||
return &datapb.AssignSegmentIDResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetSegmentInfo(ctx context.Context, in *datapb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*datapb.GetSegmentInfoResponse, error) {
|
||||
return &datapb.GetSegmentInfoResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetSegmentStates(ctx context.Context, in *datapb.GetSegmentStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentStatesResponse, error) {
|
||||
return &datapb.GetSegmentStatesResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetInsertBinlogPaths(ctx context.Context, in *datapb.GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*datapb.GetInsertBinlogPathsResponse, error) {
|
||||
return &datapb.GetInsertBinlogPathsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetCollectionStatistics(ctx context.Context, in *datapb.GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetCollectionStatisticsResponse, error) {
|
||||
return &datapb.GetCollectionStatisticsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetPartitionStatistics(ctx context.Context, in *datapb.GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetPartitionStatisticsResponse, error) {
|
||||
return &datapb.GetPartitionStatisticsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetSegmentInfoChannel(ctx context.Context, in *datapb.GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
|
||||
return &milvuspb.StringResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) SaveBinlogPaths(ctx context.Context, in *datapb.SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetRecoveryInfo(ctx context.Context, in *datapb.GetRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetRecoveryInfoResponse, error) {
|
||||
return &datapb.GetRecoveryInfoResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetFlushedSegments(ctx context.Context, in *datapb.GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*datapb.GetFlushedSegmentsResponse, error) {
|
||||
return &datapb.GetFlushedSegmentsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
|
||||
return &milvuspb.GetMetricsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) {
|
||||
return &milvuspb.ManualCompactionResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) {
|
||||
return &milvuspb.GetCompactionStateResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) {
|
||||
return &milvuspb.GetCompactionPlansResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest, opts ...grpc.CallOption) (*datapb.WatchChannelsResponse, error) {
|
||||
return &datapb.WatchChannelsResponse{}, m.Err
|
||||
}
|
||||
func (m *DataCoordClient) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) {
|
||||
return &milvuspb.GetFlushStateResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) {
|
||||
return &datapb.DropVirtualChannelResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest, opts ...grpc.CallOption) (*datapb.SetSegmentStateResponse, error) {
|
||||
return &datapb.SetSegmentStateResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
|
||||
return &datapb.ImportTaskResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *DataCoordClient) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
@ -132,3 +132,11 @@ func (m *GrpcDataCoordClient) Import(ctx context.Context, req *datapb.ImportTask
|
||||
func (m *GrpcDataCoordClient) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcDataCoordClient) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcDataCoordClient) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user