Add distributed lock for segment refgerence (#16782)

Signed-off-by: Cai.Zhang <cai.zhang@zilliz.com>
This commit is contained in:
cai.zhang 2022-05-31 16:36:03 +08:00 committed by GitHub
parent affcc2acad
commit bcf3b7426a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
43 changed files with 2701 additions and 551 deletions

View 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"
)

View File

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

View File

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

View File

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

View File

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

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View 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"
)

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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