Flush by flush channels (#26859)

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
This commit is contained in:
yihao.dai 2023-09-12 21:07:19 +08:00 committed by GitHub
parent 0ff4ddc76c
commit dd2cb1d44a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
48 changed files with 1403 additions and 721 deletions

2
go.mod
View File

@ -21,7 +21,7 @@ require (
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
github.com/klauspost/compress v1.16.5
github.com/mgutz/ansi v0.0.0-20200706080929-d51e80ef957d
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.1-0.20230905091144-d8ce91954095
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.1-0.20230907032509-23756009c643
github.com/milvus-io/milvus/pkg v0.0.1
github.com/minio/minio-go/v7 v7.0.56
github.com/prometheus/client_golang v1.14.0

6
go.sum
View File

@ -111,6 +111,8 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
github.com/bigsheeper/milvus-proto/go-api/v2 v2.0.0-20230906082705-4e84b4cf314b h1:OPGVqhJrJMOAUJeEuboKGTIsrllhJb2+ZgQMDBEdbS0=
github.com/bigsheeper/milvus-proto/go-api/v2 v2.0.0-20230906082705-4e84b4cf314b/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek=
github.com/bits-and-blooms/bitset v1.2.0 h1:Kn4yilvwNtMACtf1eYDlG8H77R07mZSPbMjLyS07ChA=
github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA=
github.com/bits-and-blooms/bloom/v3 v3.0.1 h1:Inlf0YXbgehxVjMPmCGv86iMCKMGPPrPSHtBF5yRHwA=
@ -576,8 +578,8 @@ github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/le
github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.1-0.20230905091144-d8ce91954095 h1:PyN9bVl/joOroIZKizJlDaI4wa9Zd84P6nNDm/wYUgY=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.1-0.20230905091144-d8ce91954095/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.1-0.20230907032509-23756009c643 h1:3MXEYckliGnyepZeLDrhn+speelsoRKU1IwD8JrxXMo=
github.com/milvus-io/milvus-proto/go-api/v2 v2.3.1-0.20230907032509-23756009c643/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek=
github.com/milvus-io/pulsar-client-go v0.6.10 h1:eqpJjU+/QX0iIhEo3nhOqMNXL+TyInAs1IAHZCrCM/A=
github.com/milvus-io/pulsar-client-go v0.6.10/go.mod h1:lQqCkgwDF8YFYjKA+zOheTk1tev2B+bKj5j7+nm8M1w=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=

View File

@ -106,6 +106,29 @@ func (c *Cluster) Flush(ctx context.Context, nodeID int64, channel string,
return nil
}
func (c *Cluster) FlushChannels(ctx context.Context, nodeID int64, flushTs Timestamp, channels []string) error {
if len(channels) == 0 {
return nil
}
for _, channel := range channels {
if !c.channelManager.Match(nodeID, channel) {
return fmt.Errorf("channel %s is not watched on node %d", channel, nodeID)
}
}
req := &datapb.FlushChannelsRequest{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithSourceID(paramtable.GetNodeID()),
commonpbutil.WithTargetID(nodeID),
),
FlushTs: flushTs,
Channels: channels,
}
return c.sessionManager.FlushChannels(ctx, nodeID, req)
}
// Import sends import requests to DataNodes whose ID==nodeID.
func (c *Cluster) Import(ctx context.Context, nodeID int64, it *datapb.ImportTaskRequest) {
c.sessionManager.Import(ctx, nodeID, it)

View File

@ -303,6 +303,10 @@ func (c *mockDataNodeClient) SyncSegments(ctx context.Context, req *datapb.SyncS
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) Stop() error {
c.state = commonpb.StateCode_Abnormal
return nil

View File

@ -3375,28 +3375,46 @@ func TestPostFlush(t *testing.T) {
func TestGetFlushState(t *testing.T) {
t.Run("get flush state with all flushed segments", func(t *testing.T) {
svr := &Server{
meta: &meta{
segments: &SegmentsInfo{
segments: map[int64]*SegmentInfo{
1: {
SegmentInfo: &datapb.SegmentInfo{
ID: 1,
State: commonpb.SegmentState_Flushed,
},
},
2: {
SegmentInfo: &datapb.SegmentInfo{
ID: 2,
State: commonpb.SegmentState_Flushed,
},
},
},
meta, err := newMemoryMeta()
assert.NoError(t, err)
svr := newTestServerWithMeta(t, nil, meta)
defer closeTestServer(t, svr)
err = meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: 1,
State: commonpb.SegmentState_Flushed,
},
})
assert.NoError(t, err)
err = meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: 2,
State: commonpb.SegmentState_Flushed,
},
})
assert.NoError(t, err)
var (
vchannel = "ch1"
collection = int64(0)
)
svr.channelManager = &ChannelManager{
store: &ChannelStore{
channelsInfo: map[int64]*NodeChannelInfo{
1: {NodeID: 1, Channels: []*channel{{Name: vchannel, CollectionID: collection}}},
},
},
}
svr.stateCode.Store(commonpb.StateCode_Healthy)
resp, err := svr.GetFlushState(context.TODO(), &milvuspb.GetFlushStateRequest{SegmentIDs: []int64{1, 2}})
err = svr.meta.UpdateChannelCheckpoint(vchannel, &msgpb.MsgPosition{
MsgID: []byte{1},
Timestamp: 12,
})
assert.NoError(t, err)
resp, err := svr.GetFlushState(context.TODO(), &datapb.GetFlushStateRequest{SegmentIDs: []int64{1, 2}})
assert.NoError(t, err)
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
@ -3405,29 +3423,46 @@ func TestGetFlushState(t *testing.T) {
})
t.Run("get flush state with unflushed segments", func(t *testing.T) {
svr := &Server{
meta: &meta{
segments: &SegmentsInfo{
segments: map[int64]*SegmentInfo{
1: {
SegmentInfo: &datapb.SegmentInfo{
ID: 1,
State: commonpb.SegmentState_Flushed,
},
},
2: {
SegmentInfo: &datapb.SegmentInfo{
ID: 2,
State: commonpb.SegmentState_Sealed,
},
},
},
meta, err := newMemoryMeta()
assert.NoError(t, err)
svr := newTestServerWithMeta(t, nil, meta)
defer closeTestServer(t, svr)
err = meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: 1,
State: commonpb.SegmentState_Flushed,
},
})
assert.NoError(t, err)
err = meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: 2,
State: commonpb.SegmentState_Sealed,
},
})
assert.NoError(t, err)
var (
vchannel = "ch1"
collection = int64(0)
)
svr.channelManager = &ChannelManager{
store: &ChannelStore{
channelsInfo: map[int64]*NodeChannelInfo{
1: {NodeID: 1, Channels: []*channel{{Name: vchannel, CollectionID: collection}}},
},
},
}
svr.stateCode.Store(commonpb.StateCode_Healthy)
resp, err := svr.GetFlushState(context.TODO(), &milvuspb.GetFlushStateRequest{SegmentIDs: []int64{1, 2}})
err = svr.meta.UpdateChannelCheckpoint(vchannel, &msgpb.MsgPosition{
MsgID: []byte{1},
Timestamp: 12,
})
assert.NoError(t, err)
resp, err := svr.GetFlushState(context.TODO(), &datapb.GetFlushStateRequest{SegmentIDs: []int64{1, 2}})
assert.NoError(t, err)
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
@ -3436,35 +3471,145 @@ func TestGetFlushState(t *testing.T) {
})
t.Run("get flush state with compacted segments", func(t *testing.T) {
svr := &Server{
meta: &meta{
segments: &SegmentsInfo{
segments: map[int64]*SegmentInfo{
1: {
SegmentInfo: &datapb.SegmentInfo{
ID: 1,
State: commonpb.SegmentState_Flushed,
},
},
2: {
SegmentInfo: &datapb.SegmentInfo{
ID: 2,
State: commonpb.SegmentState_Dropped,
},
},
},
meta, err := newMemoryMeta()
assert.NoError(t, err)
svr := newTestServerWithMeta(t, nil, meta)
defer closeTestServer(t, svr)
err = meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: 1,
State: commonpb.SegmentState_Flushed,
},
})
assert.NoError(t, err)
err = meta.AddSegment(&SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: 2,
State: commonpb.SegmentState_Dropped,
},
})
assert.NoError(t, err)
var (
vchannel = "ch1"
collection = int64(0)
)
svr.channelManager = &ChannelManager{
store: &ChannelStore{
channelsInfo: map[int64]*NodeChannelInfo{
1: {NodeID: 1, Channels: []*channel{{Name: vchannel, CollectionID: collection}}},
},
},
}
svr.stateCode.Store(commonpb.StateCode_Healthy)
resp, err := svr.GetFlushState(context.TODO(), &milvuspb.GetFlushStateRequest{SegmentIDs: []int64{1, 2}})
err = svr.meta.UpdateChannelCheckpoint(vchannel, &msgpb.MsgPosition{
MsgID: []byte{1},
Timestamp: 12,
})
assert.NoError(t, err)
resp, err := svr.GetFlushState(context.TODO(), &datapb.GetFlushStateRequest{SegmentIDs: []int64{1, 2}})
assert.NoError(t, err)
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Flushed: true,
}, resp)
})
t.Run("channel flushed", func(t *testing.T) {
meta, err := newMemoryMeta()
assert.NoError(t, err)
svr := newTestServerWithMeta(t, nil, meta)
defer closeTestServer(t, svr)
var (
vchannel = "ch1"
collection = int64(0)
)
svr.channelManager = &ChannelManager{
store: &ChannelStore{
channelsInfo: map[int64]*NodeChannelInfo{
1: {NodeID: 1, Channels: []*channel{{Name: vchannel, CollectionID: collection}}},
},
},
}
err = svr.meta.UpdateChannelCheckpoint(vchannel, &msgpb.MsgPosition{
MsgID: []byte{1},
Timestamp: 12,
})
assert.NoError(t, err)
resp, err := svr.GetFlushState(context.Background(), &datapb.GetFlushStateRequest{
FlushTs: 11,
CollectionID: collection,
})
assert.NoError(t, err)
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Flushed: true,
}, resp)
})
t.Run("channel unflushed", func(t *testing.T) {
meta, err := newMemoryMeta()
assert.NoError(t, err)
svr := newTestServerWithMeta(t, nil, meta)
defer closeTestServer(t, svr)
var (
vchannel = "ch1"
collection = int64(0)
)
svr.channelManager = &ChannelManager{
store: &ChannelStore{
channelsInfo: map[int64]*NodeChannelInfo{
1: {NodeID: 1, Channels: []*channel{{Name: vchannel, CollectionID: collection}}},
},
},
}
err = svr.meta.UpdateChannelCheckpoint(vchannel, &msgpb.MsgPosition{
MsgID: []byte{1},
Timestamp: 10,
})
assert.NoError(t, err)
resp, err := svr.GetFlushState(context.Background(), &datapb.GetFlushStateRequest{
FlushTs: 11,
CollectionID: collection,
})
assert.NoError(t, err)
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Flushed: false,
}, resp)
})
t.Run("no channels", func(t *testing.T) {
meta, err := newMemoryMeta()
assert.NoError(t, err)
svr := newTestServerWithMeta(t, nil, meta)
defer closeTestServer(t, svr)
var (
collection = int64(0)
)
resp, err := svr.GetFlushState(context.Background(), &datapb.GetFlushStateRequest{
FlushTs: 11,
CollectionID: collection,
})
assert.NoError(t, err)
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Flushed: false,
}, resp)
})
}
func TestGetFlushAllState(t *testing.T) {

View File

@ -42,6 +42,7 @@ import (
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
@ -116,24 +117,46 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
flushSegmentIDs := make([]UniqueID, 0, len(segments))
for _, segment := range segments {
if segment != nil &&
(segment.GetState() == commonpb.SegmentState_Flushed ||
segment.GetState() == commonpb.SegmentState_Flushing) &&
(isFlushState(segment.GetState())) &&
!sealedSegmentsIDDict[segment.GetID()] {
flushSegmentIDs = append(flushSegmentIDs, segment.GetID())
}
}
err = retry.Do(ctx, func() error {
for _, channelInfo := range s.channelManager.GetChannels() {
nodeID := channelInfo.NodeID
channels := lo.Filter(channelInfo.Channels, func(channel *channel, _ int) bool {
return channel.CollectionID == req.GetCollectionID()
})
channelNames := lo.Map(channels, func(channel *channel, _ int) string {
return channel.Name
})
err = s.cluster.FlushChannels(ctx, nodeID, ts, channelNames)
if err != nil {
return err
}
}
return nil
})
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
log.Info("flush response with segments",
zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64s("sealSegments", sealedSegmentIDs),
zap.Int64s("flushSegments", flushSegmentIDs),
zap.Time("timeOfSeal", timeOfSeal))
zap.Time("timeOfSeal", timeOfSeal),
zap.Time("flushTs", tsoutil.PhysicalTime(ts)))
resp.Status.ErrorCode = commonpb.ErrorCode_Success
resp.DbID = req.GetDbID()
resp.CollectionID = req.GetCollectionID()
resp.SegmentIDs = sealedSegmentIDs
resp.TimeOfSeal = timeOfSeal.Unix()
resp.FlushSegmentIDs = flushSegmentIDs
resp.FlushTs = ts
return resp, nil
}
@ -1245,36 +1268,70 @@ func (s *Server) WatchChannels(ctx context.Context, req *datapb.WatchChannelsReq
return resp, nil
}
// GetFlushState gets the flush state of multiple segments
func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
log := log.Ctx(ctx).WithRateGroup("dc.GetFlushState", 1, 60)
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
func (s *Server) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
log := log.Ctx(ctx).With(zap.Int64("collection", req.GetCollectionID()),
zap.Time("flushTs", tsoutil.PhysicalTime(req.GetFlushTs()))).
WithRateGroup("dc.GetFlushState", 1, 60)
resp := &milvuspb.GetFlushStateResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}}
if s.isClosed() {
log.Warn("DataCoord receive GetFlushState request, server closed",
zap.Int64s("segmentIDs", req.GetSegmentIDs()), zap.Int("len", len(req.GetSegmentIDs())))
log.Warn("DataCoord receive GetFlushState request, server closed")
resp.Status.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return resp, nil
}
var unflushed []UniqueID
for _, sid := range req.GetSegmentIDs() {
segment := s.meta.GetHealthySegment(sid)
// segment is nil if it was compacted or it's a empty segment and is set to dropped
if segment == nil || segment.GetState() == commonpb.SegmentState_Flushing ||
segment.GetState() == commonpb.SegmentState_Flushed {
continue
if len(req.GetSegmentIDs()) > 0 {
var unflushed []UniqueID
for _, sid := range req.GetSegmentIDs() {
segment := s.meta.GetHealthySegment(sid)
// segment is nil if it was compacted, or it's an empty segment and is set to dropped
if segment == nil || isFlushState(segment.GetState()) {
continue
}
unflushed = append(unflushed, sid)
}
if len(unflushed) != 0 {
log.RatedInfo(10, "DataCoord receive GetFlushState request, Flushed is false", zap.Int64s("unflushed", unflushed), zap.Int("len", len(unflushed)))
resp.Flushed = false
resp.Status.ErrorCode = commonpb.ErrorCode_Success
return resp, nil
}
unflushed = append(unflushed, sid)
}
if len(unflushed) != 0 {
log.RatedInfo(10, "DataCoord receive GetFlushState request, Flushed is false", zap.Int64s("unflushed", unflushed), zap.Int("len", len(unflushed)))
resp.Flushed = false
} else {
log.Info("DataCoord receive GetFlushState request, Flushed is true", zap.Int64s("segmentIDs", req.GetSegmentIDs()), zap.Int("len", len(req.GetSegmentIDs())))
resp.Flushed = true
channels := make([]string, 0)
for _, channelInfo := range s.channelManager.GetChannels() {
filtered := lo.Filter(channelInfo.Channels, func(channel *channel, _ int) bool {
return channel.CollectionID == req.GetCollectionID()
})
channelNames := lo.Map(filtered, func(channel *channel, _ int) string {
return channel.Name
})
channels = append(channels, channelNames...)
}
if len(channels) == 0 {
resp.Flushed = false
resp.Status.ErrorCode = commonpb.ErrorCode_Success
log.Warn("GetFlushState failed, no channels found")
return resp, nil
}
for _, channel := range channels {
cp := s.meta.GetChannelCheckpoint(channel)
if cp == nil || cp.GetTimestamp() < req.GetFlushTs() {
resp.Flushed = false
resp.Status.ErrorCode = commonpb.ErrorCode_Success
log.RatedInfo(10, "GetFlushState failed, channel unflushed", zap.String("channel", channel),
zap.Time("CP", tsoutil.PhysicalTime(cp.GetTimestamp())),
zap.Duration("lag", tsoutil.PhysicalTime(req.GetFlushTs()).Sub(tsoutil.PhysicalTime(cp.GetTimestamp()))))
return resp, nil
}
}
resp.Flushed = true
resp.Status.ErrorCode = commonpb.ErrorCode_Success
log.Info("GetFlushState all flushed")
return resp, nil
}

View File

@ -300,6 +300,24 @@ func (c *SessionManager) GetCompactionState() map[int64]*datapb.CompactionStateR
return rst
}
func (c *SessionManager) FlushChannels(ctx context.Context, nodeID int64, req *datapb.FlushChannelsRequest) error {
log := log.Ctx(ctx).With(zap.Int64("nodeID", nodeID))
cli, err := c.getClient(ctx, nodeID)
if err != nil {
log.Warn("failed to get client", zap.Error(err))
return err
}
resp, err := cli.FlushChannels(ctx, req)
err = VerifyResponse(resp, err)
if err != nil {
log.Warn("SessionManager.FlushChannels failed", zap.Error(err))
return err
}
log.Info("SessionManager.FlushChannels successfully")
return nil
}
func (c *SessionManager) getClient(ctx context.Context, nodeID int64) (types.DataNode, error) {
c.sessions.RLock()
session, ok := c.sessions.data[nodeID]

View File

@ -99,7 +99,11 @@ type Channel interface {
// getTotalMemorySize returns the sum of memory sizes of segments.
getTotalMemorySize() int64
forceToSync()
setIsHighMemory(b bool)
getIsHighMemory() bool
getFlushTs() Timestamp
setFlushTs(ts Timestamp)
close()
}
@ -114,7 +118,15 @@ type ChannelMeta struct {
segMu sync.RWMutex
segments map[UniqueID]*Segment
needToSync *atomic.Bool
// isHighMemory is intended to trigger the syncing of segments
// when segment's buffer consumes a significant amount of memory.
isHighMemory *atomic.Bool
// flushTs is intended to trigger:
// 1. the syncing of segments when consumed ts exceeds flushTs;
// 2. the updating of channelCP when channelCP exceeds flushTs.
flushTs *atomic.Uint64
syncPolicies []segmentSyncPolicy
metaService *metaService
@ -147,10 +159,12 @@ func newChannel(channelName string, collID UniqueID, schema *schemapb.Collection
segments: make(map[UniqueID]*Segment),
needToSync: atomic.NewBool(false),
isHighMemory: atomic.NewBool(false),
flushTs: atomic.NewUint64(math.MaxUint64),
syncPolicies: []segmentSyncPolicy{
syncPeriodically(),
syncMemoryTooHigh(),
syncSegmentsAtTs(),
},
metaService: metaService,
@ -279,7 +293,7 @@ func (c *ChannelMeta) listSegmentIDsToSync(ts Timestamp) []UniqueID {
segIDsToSync := typeutil.NewUniqueSet()
for _, policy := range c.syncPolicies {
segments := policy(validSegs, ts, c.needToSync)
segments := policy(validSegs, c, ts)
for _, segID := range segments {
segIDsToSync.Insert(segID)
}
@ -797,6 +811,7 @@ func (c *ChannelMeta) listNotFlushedSegmentIDs() []UniqueID {
}
func (c *ChannelMeta) getChannelCheckpoint(ttPos *msgpb.MsgPosition) *msgpb.MsgPosition {
log := log.With().WithRateGroup("ChannelMeta", 1, 60)
c.segMu.RLock()
defer c.segMu.RUnlock()
channelCP := &msgpb.MsgPosition{Timestamp: math.MaxUint64}
@ -818,8 +833,7 @@ func (c *ChannelMeta) getChannelCheckpoint(ttPos *msgpb.MsgPosition) *msgpb.MsgP
channelCP = db.startPos
}
}
// TODO: maybe too many logs would print
log.Debug("getChannelCheckpoint for segment", zap.Int64("segmentID", seg.segmentID),
log.RatedDebug(10, "getChannelCheckpoint for segment", zap.Int64("segmentID", seg.segmentID),
zap.Bool("isCurIBEmpty", seg.curInsertBuf == nil),
zap.Bool("isCurDBEmpty", seg.curDeleteBuf == nil),
zap.Int("len(hisIB)", len(seg.historyInsertBuf)),
@ -926,8 +940,12 @@ func (c *ChannelMeta) evictHistoryDeleteBuffer(segmentID UniqueID, endPos *msgpb
log.Warn("cannot find segment when evictHistoryDeleteBuffer", zap.Int64("segmentID", segmentID))
}
func (c *ChannelMeta) forceToSync() {
c.needToSync.Store(true)
func (c *ChannelMeta) setIsHighMemory(b bool) {
c.isHighMemory.Store(b)
}
func (c *ChannelMeta) getIsHighMemory() bool {
return c.isHighMemory.Load()
}
func (c *ChannelMeta) getTotalMemorySize() int64 {
@ -940,6 +958,14 @@ func (c *ChannelMeta) getTotalMemorySize() int64 {
return res
}
func (c *ChannelMeta) getFlushTs() Timestamp {
return c.flushTs.Load()
}
func (c *ChannelMeta) setFlushTs(ts Timestamp) {
c.flushTs.Store(ts)
}
func (c *ChannelMeta) close() {
c.closed.Store(true)
}

View File

@ -20,7 +20,9 @@ import (
"bytes"
"context"
"encoding/binary"
"fmt"
"math"
"math/rand"
"os"
"testing"
"time"
@ -203,7 +205,7 @@ func TestDataSyncService_Start(t *testing.T) {
defer cancel()
// init data node
insertChannelName := "by-dev-rootcoord-dml"
insertChannelName := fmt.Sprintf("by-dev-rootcoord-dml-%d", rand.Int())
Factory := &MetaFactory{}
collMeta := Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
@ -288,7 +290,7 @@ func TestDataSyncService_Start(t *testing.T) {
timeRange := TimeRange{
timestampMin: 0,
timestampMax: math.MaxUint64,
timestampMax: math.MaxUint64 - 1,
}
dataFactory := NewDataFactory()
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, tsoutil.GetCurrentTime())
@ -318,7 +320,7 @@ func TestDataSyncService_Start(t *testing.T) {
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
MsgID: UniqueID(0),
Timestamp: math.MaxUint64,
Timestamp: math.MaxUint64 - 1,
SourceID: 0,
},
},

View File

@ -451,7 +451,7 @@ func (ibNode *insertBufferNode) FillInSyncTasks(fgMsg *flowGraphMsg, seg2Upload
func (ibNode *insertBufferNode) Sync(fgMsg *flowGraphMsg, seg2Upload []UniqueID, endPosition *msgpb.MsgPosition) []UniqueID {
syncTasks := ibNode.FillInSyncTasks(fgMsg, seg2Upload)
segmentsToSync := make([]UniqueID, 0, len(syncTasks))
ibNode.channel.(*ChannelMeta).needToSync.Store(false)
ibNode.channel.setIsHighMemory(false)
for _, task := range syncTasks {
log := log.With(zap.Int64("segmentID", task.segmentID),

View File

@ -343,7 +343,7 @@ func TestFlowGraphInsertBufferNode_AutoFlush(t *testing.T) {
channel := &ChannelMeta{
collectionID: collMeta.ID,
segments: make(map[UniqueID]*Segment),
needToSync: atomic.NewBool(false),
isHighMemory: atomic.NewBool(false),
}
channel.metaService = newMetaService(mockRootCoord, collMeta.ID)
@ -590,7 +590,7 @@ func TestInsertBufferNodeRollBF(t *testing.T) {
channel := &ChannelMeta{
collectionID: collMeta.ID,
segments: make(map[UniqueID]*Segment),
needToSync: atomic.NewBool(false),
isHighMemory: atomic.NewBool(false),
}
channel.metaService = newMetaService(mockRootCoord, collMeta.ID)

View File

@ -108,7 +108,7 @@ func (fm *flowgraphManager) execute(totalMemory uint64) {
return channels[i].bufferSize > channels[j].bufferSize
})
if fg, ok := fm.flowgraphs.Get(channels[0].channel); ok { // sync the first channel with the largest memory usage
fg.channel.forceToSync()
fg.channel.setIsHighMemory(true)
log.Info("notify flowgraph to sync",
zap.String("channel", channels[0].channel), zap.Int64("bufferSize", channels[0].bufferSize))
}

View File

@ -222,14 +222,14 @@ func TestFlowGraphManager(t *testing.T) {
err = fg.channel.addSegment(addSegmentReq{segID: 0})
assert.NoError(t, err)
fg.channel.getSegment(0).memorySize = memorySize
fg.channel.(*ChannelMeta).needToSync.Store(false)
fg.channel.setIsHighMemory(false)
}
fm.execute(test.totalMemory)
for i, needToSync := range test.expectNeedToSync {
vchannel := fmt.Sprintf("%s%d", channelPrefix, i)
fg, ok := fm.flowgraphs.Get(vchannel)
assert.True(t, ok)
assert.Equal(t, needToSync, fg.channel.(*ChannelMeta).needToSync.Load())
assert.Equal(t, needToSync, fg.channel.getIsHighMemory())
}
}
})

View File

@ -19,6 +19,7 @@ package datanode
import (
"context"
"fmt"
"math"
"reflect"
"time"
@ -73,30 +74,40 @@ func (ttn *ttNode) Operate(in []Msg) []Msg {
fgMsg := in[0].(*flowGraphMsg)
if fgMsg.IsCloseMsg() {
if len(fgMsg.endPositions) > 0 {
channelPos := ttn.channel.getChannelCheckpoint(fgMsg.endPositions[0])
log.Info("flowgraph is closing, force update channel CP",
zap.Uint64("endTs", fgMsg.endPositions[0].GetTimestamp()),
zap.String("channel", fgMsg.endPositions[0].GetChannelName()))
ttn.updateChannelCP(fgMsg.endPositions[0])
zap.Time("cpTs", tsoutil.PhysicalTime(channelPos.GetTimestamp())),
zap.String("channel", channelPos.GetChannelName()))
ttn.updateChannelCP(channelPos)
}
return in
}
curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax)
channelPos := ttn.channel.getChannelCheckpoint(fgMsg.endPositions[0])
log := log.With(zap.String("channel", ttn.vChannelName),
zap.Time("cpTs", tsoutil.PhysicalTime(channelPos.GetTimestamp())))
if curTs.Sub(ttn.lastUpdateTime) >= updateChanCPInterval {
ttn.updateChannelCP(fgMsg.endPositions[0])
ttn.lastUpdateTime = curTs
if err := ttn.updateChannelCP(channelPos); err == nil {
ttn.lastUpdateTime = curTs
log.Info("update channel cp periodically")
return []Msg{}
}
}
if channelPos.GetTimestamp() >= ttn.channel.getFlushTs() {
if err := ttn.updateChannelCP(channelPos); err == nil {
ttn.lastUpdateTime = curTs
log.Info("update channel cp at updateTs", zap.Time("updateTs", tsoutil.PhysicalTime(ttn.channel.getFlushTs())))
ttn.channel.setFlushTs(math.MaxUint64)
}
}
return []Msg{}
}
func (ttn *ttNode) updateChannelCP(ttPos *msgpb.MsgPosition) {
channelPos := ttn.channel.getChannelCheckpoint(ttPos)
if channelPos == nil || channelPos.MsgID == nil {
log.Warn("updateChannelCP failed, get nil check point", zap.String("vChannel", ttn.vChannelName))
return
}
channelCPTs, _ := tsoutil.ParseTS(channelPos.Timestamp)
func (ttn *ttNode) updateChannelCP(channelPos *msgpb.MsgPosition) error {
channelCPTs, _ := tsoutil.ParseTS(channelPos.GetTimestamp())
ctx, cancel := context.WithTimeout(context.Background(), updateChanCPTimeout)
defer cancel()
@ -110,13 +121,14 @@ func (ttn *ttNode) updateChannelCP(ttPos *msgpb.MsgPosition) {
if err = funcutil.VerifyResponse(resp, err); err != nil {
log.Warn("UpdateChannelCheckpoint failed", zap.String("channel", ttn.vChannelName),
zap.Time("channelCPTs", channelCPTs), zap.Error(err))
return
return err
}
log.Info("UpdateChannelCheckpoint success",
zap.String("channel", ttn.vChannelName),
zap.Uint64("cpTs", channelPos.Timestamp),
zap.Uint64("cpTs", channelPos.GetTimestamp()),
zap.Time("cpTime", channelCPTs))
return nil
}
func newTTNode(config *nodeConfig, dc types.DataCoord) (*ttNode, error) {

View File

@ -23,18 +23,18 @@ import (
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"go.uber.org/atomic"
"github.com/samber/lo"
"go.uber.org/zap"
)
const minSyncSize = 0.5 * 1024 * 1024
// segmentsSyncPolicy sync policy applies to segments
type segmentSyncPolicy func(segments []*Segment, ts Timestamp, needToSync *atomic.Bool) []UniqueID
type segmentSyncPolicy func(segments []*Segment, c Channel, ts Timestamp) []UniqueID
// syncPeriodically get segmentSyncPolicy with segments sync periodically.
func syncPeriodically() segmentSyncPolicy {
return func(segments []*Segment, ts Timestamp, _ *atomic.Bool) []UniqueID {
return func(segments []*Segment, c Channel, ts Timestamp) []UniqueID {
segmentsToSync := make([]UniqueID, 0)
for _, seg := range segments {
endPosTime := tsoutil.PhysicalTime(ts)
@ -45,7 +45,7 @@ func syncPeriodically() segmentSyncPolicy {
}
}
if len(segmentsToSync) > 0 {
log.Info("sync segment periodically", zap.Int64s("segmentID", segmentsToSync))
log.Info("sync segment periodically", zap.Int64s("segmentIDs", segmentsToSync))
}
return segmentsToSync
}
@ -53,8 +53,8 @@ func syncPeriodically() segmentSyncPolicy {
// syncMemoryTooHigh force sync the largest segment.
func syncMemoryTooHigh() segmentSyncPolicy {
return func(segments []*Segment, ts Timestamp, needToSync *atomic.Bool) []UniqueID {
if len(segments) == 0 || !needToSync.Load() {
return func(segments []*Segment, c Channel, _ Timestamp) []UniqueID {
if len(segments) == 0 || !c.getIsHighMemory() {
return nil
}
sort.Slice(segments, func(i, j int) bool {
@ -74,3 +74,22 @@ func syncMemoryTooHigh() segmentSyncPolicy {
return syncSegments
}
}
// syncSegmentsAtTs returns a new segmentSyncPolicy, sync segments when ts exceeds ChannelMeta.flushTs
func syncSegmentsAtTs() segmentSyncPolicy {
return func(segments []*Segment, c Channel, ts Timestamp) []UniqueID {
flushTs := c.getFlushTs()
if flushTs != 0 && ts >= flushTs {
segmentsWithBuffer := lo.Filter(segments, func(segment *Segment, _ int) bool {
return !segment.isBufferEmpty()
})
segmentIDs := lo.Map(segmentsWithBuffer, func(segment *Segment, _ int) UniqueID {
return segment.segmentID
})
log.Info("sync segment at ts", zap.Int64s("segmentIDs", segmentIDs),
zap.Time("ts", tsoutil.PhysicalTime(ts)), zap.Time("flushTs", tsoutil.PhysicalTime(flushTs)))
return segmentIDs
}
return nil
}
}

View File

@ -18,14 +18,13 @@ package datanode
import (
"fmt"
"math"
"testing"
"time"
"github.com/stretchr/testify/assert"
"go.uber.org/atomic"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/stretchr/testify/assert"
)
func TestSyncPeriodically(t *testing.T) {
@ -55,7 +54,7 @@ func TestSyncPeriodically(t *testing.T) {
if test.isBufferEmpty {
segment.curInsertBuf = nil
}
res := policy([]*Segment{segment}, tsoutil.ComposeTSByTime(test.endPosTs, 0), nil)
res := policy([]*Segment{segment}, nil, tsoutil.ComposeTSByTime(test.endPosTs, 0))
assert.Equal(t, test.shouldSyncNum, len(res))
})
}
@ -65,7 +64,7 @@ func TestSyncMemoryTooHigh(t *testing.T) {
tests := []struct {
testName string
syncSegmentNum int
needToSync bool
isHighMemory bool
memorySizesInMB []float64
shouldSyncSegs []UniqueID
}{
@ -75,7 +74,7 @@ func TestSyncMemoryTooHigh(t *testing.T) {
[]float64{1, 2, 3, 4, 5}, []UniqueID{5, 4}},
{"test normal 3", 5, true,
[]float64{1, 2, 3, 4, 5}, []UniqueID{5, 4, 3, 2, 1}},
{"test needToSync false", 3, false,
{"test isHighMemory false", 3, false,
[]float64{1, 2, 3, 4, 5}, []UniqueID{}},
{"test syncSegmentNum 1", 1, true,
[]float64{1, 2, 3, 4, 5}, []UniqueID{5}},
@ -85,6 +84,8 @@ func TestSyncMemoryTooHigh(t *testing.T) {
for _, test := range tests {
t.Run(test.testName, func(t *testing.T) {
channel := newChannel("channel", 0, nil, nil, nil)
channel.setIsHighMemory(test.isHighMemory)
Params.Save(Params.DataNodeCfg.MemoryForceSyncSegmentNum.Key, fmt.Sprintf("%d", test.syncSegmentNum))
policy := syncMemoryTooHigh()
segments := make([]*Segment, len(test.memorySizesInMB))
@ -93,8 +94,39 @@ func TestSyncMemoryTooHigh(t *testing.T) {
segmentID: UniqueID(i + 1), memorySize: int64(test.memorySizesInMB[i] * 1024 * 1024),
}
}
segs := policy(segments, 0, atomic.NewBool(test.needToSync))
segs := policy(segments, channel, 0)
assert.ElementsMatch(t, segs, test.shouldSyncSegs)
})
}
}
func TestSyncSegmentsAtTs(t *testing.T) {
tests := []struct {
testName string
ts Timestamp
flushTs Timestamp
shouldSyncNum int
}{
{"test ts < flushTs", 100, 200, 0},
{"test ts > flushTs", 300, 200, 1},
{"test ts = flushTs", 100, 100, 1},
{"test flushTs = 0", 100, 0, 0},
{"test flushTs = maxUint64", 100, math.MaxUint64, 0},
}
for _, test := range tests {
t.Run(test.testName, func(t *testing.T) {
channel := newChannel("channel", 0, nil, nil, nil)
channel.setFlushTs(test.flushTs)
segment := &Segment{}
segment.setInsertBuffer(&BufferData{
startPos: &msgpb.MsgPosition{},
})
policy := syncSegmentsAtTs()
res := policy([]*Segment{segment}, channel, test.ts)
assert.Equal(t, test.shouldSyncNum, len(res))
})
}
}

View File

@ -520,6 +520,30 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
return resp, nil
}
func (node *DataNode) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
log := log.Ctx(ctx).With(zap.Int64("nodeId", paramtable.GetNodeID()),
zap.Time("flushTs", tsoutil.PhysicalTime(req.GetFlushTs())),
zap.Int("len(channels)", len(req.GetChannels())))
log.Info("DataNode receives FlushChannels request")
if !node.isHealthy() {
err := merr.WrapErrServiceNotReady(node.GetStateCode().String())
log.Warn("DataNode.FlushChannels failed", zap.Error(err))
return merr.Status(err), nil
}
for _, channel := range req.GetChannels() {
fg, ok := node.flowgraphManager.getFlowgraphService(channel)
if !ok {
return merr.Status(merr.WrapErrChannelNotFound(channel)), nil
}
fg.channel.setFlushTs(req.GetFlushTs())
}
return merr.Status(nil), nil
}
func (node *DataNode) getPartitions(ctx context.Context, dbName string, collectionName string) (map[string]int64, error) {
req := &milvuspb.ShowPartitionsRequest{
Base: commonpbutil.NewMsgBase(

View File

@ -807,3 +807,36 @@ func (s *DataNodeServicesSuite) TestResendSegmentStats() {
s.Assert().True(merr.Ok(resp.GetStatus()))
s.Assert().ElementsMatch([]UniqueID{0, 1, 2}, resp.GetSegResent())
}
func (s *DataNodeServicesSuite) TestFlushChannels() {
dmChannelName := "fake-by-dev-rootcoord-dml-channel-TestFlushChannels"
vChan := &datapb.VchannelInfo{
CollectionID: 1,
ChannelName: dmChannelName,
UnflushedSegmentIds: []int64{},
FlushedSegmentIds: []int64{},
}
err := s.node.flowgraphManager.addAndStart(s.node, vChan, nil, genTestTickler())
s.Require().NoError(err)
fgService, ok := s.node.flowgraphManager.getFlowgraphService(dmChannelName)
s.Require().True(ok)
flushTs := Timestamp(100)
req := &datapb.FlushChannelsRequest{
Base: &commonpb.MsgBase{
TargetID: s.node.GetSession().ServerID,
},
FlushTs: flushTs,
Channels: []string{dmChannelName},
}
status, err := s.node.FlushChannels(s.ctx, req)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(status))
s.Assert().True(fgService.channel.getFlushTs() == flushTs)
}

View File

@ -442,8 +442,8 @@ func (c *Client) WatchChannels(ctx context.Context, req *datapb.WatchChannelsReq
})
}
// GetFlushState gets the flush state of multiple segments
func (c *Client) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
func (c *Client) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.GetFlushStateResponse, error) {
return client.GetFlushState(ctx, req)
})

View File

@ -368,8 +368,8 @@ func (s *Server) WatchChannels(ctx context.Context, req *datapb.WatchChannelsReq
return s.dataCoord.WatchChannels(ctx, req)
}
// GetFlushState gets the flush state of multiple segments
func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
func (s *Server) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return s.dataCoord.GetFlushState(ctx, req)
}

View File

@ -207,7 +207,7 @@ func (m *MockDataCoord) WatchChannels(ctx context.Context, req *datapb.WatchChan
return m.watchChannelsResp, m.err
}
func (m *MockDataCoord) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
func (m *MockDataCoord) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return m.getFlushStateResp, m.err
}

View File

@ -227,3 +227,10 @@ func (c *Client) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsReque
return client.SyncSegments(ctx, req)
})
}
// FlushChannels notifies DataNode to sync all the segments belongs to the target channels.
func (c *Client) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
return client.FlushChannels(ctx, req)
})
}

View File

@ -400,3 +400,7 @@ func (s *Server) AddImportSegment(ctx context.Context, request *datapb.AddImport
func (s *Server) SyncSegments(ctx context.Context, request *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
return s.datanode.SyncSegments(ctx, request)
}
func (s *Server) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
return s.datanode.FlushChannels(ctx, req)
}

View File

@ -148,6 +148,10 @@ func (m *MockDataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegment
return m.status, m.err
}
func (m *MockDataNode) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
return m.status, m.err
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type mockDataCoord struct {
types.DataCoord

View File

@ -924,7 +924,7 @@ func (s *Server) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.
return s.proxy.GetCompactionStateWithPlans(ctx, req)
}
// GetFlushState gets the flush state of multiple segments
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return s.proxy.GetFlushState(ctx, req)
}

View File

@ -430,7 +430,7 @@ func (m *MockDataCoord) WatchChannels(ctx context.Context, req *datapb.WatchChan
return nil, nil
}
func (m *MockDataCoord) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
func (m *MockDataCoord) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return nil, nil
}

View File

@ -68,8 +68,8 @@ type MockDataCoord_AssignSegmentID_Call struct {
}
// AssignSegmentID is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.AssignSegmentIDRequest
// - ctx context.Context
// - req *datapb.AssignSegmentIDRequest
func (_e *MockDataCoord_Expecter) AssignSegmentID(ctx interface{}, req interface{}) *MockDataCoord_AssignSegmentID_Call {
return &MockDataCoord_AssignSegmentID_Call{Call: _e.mock.On("AssignSegmentID", ctx, req)}
}
@ -123,8 +123,8 @@ type MockDataCoord_BroadcastAlteredCollection_Call struct {
}
// BroadcastAlteredCollection is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.AlterCollectionRequest
// - ctx context.Context
// - req *datapb.AlterCollectionRequest
func (_e *MockDataCoord_Expecter) BroadcastAlteredCollection(ctx interface{}, req interface{}) *MockDataCoord_BroadcastAlteredCollection_Call {
return &MockDataCoord_BroadcastAlteredCollection_Call{Call: _e.mock.On("BroadcastAlteredCollection", ctx, req)}
}
@ -178,8 +178,8 @@ type MockDataCoord_CheckHealth_Call struct {
}
// CheckHealth is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.CheckHealthRequest
// - ctx context.Context
// - req *milvuspb.CheckHealthRequest
func (_e *MockDataCoord_Expecter) CheckHealth(ctx interface{}, req interface{}) *MockDataCoord_CheckHealth_Call {
return &MockDataCoord_CheckHealth_Call{Call: _e.mock.On("CheckHealth", ctx, req)}
}
@ -233,8 +233,8 @@ type MockDataCoord_CreateIndex_Call struct {
}
// CreateIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.CreateIndexRequest
// - ctx context.Context
// - req *indexpb.CreateIndexRequest
func (_e *MockDataCoord_Expecter) CreateIndex(ctx interface{}, req interface{}) *MockDataCoord_CreateIndex_Call {
return &MockDataCoord_CreateIndex_Call{Call: _e.mock.On("CreateIndex", ctx, req)}
}
@ -288,8 +288,8 @@ type MockDataCoord_DescribeIndex_Call struct {
}
// DescribeIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.DescribeIndexRequest
// - ctx context.Context
// - req *indexpb.DescribeIndexRequest
func (_e *MockDataCoord_Expecter) DescribeIndex(ctx interface{}, req interface{}) *MockDataCoord_DescribeIndex_Call {
return &MockDataCoord_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex", ctx, req)}
}
@ -343,8 +343,8 @@ type MockDataCoord_DropIndex_Call struct {
}
// DropIndex is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.DropIndexRequest
// - ctx context.Context
// - req *indexpb.DropIndexRequest
func (_e *MockDataCoord_Expecter) DropIndex(ctx interface{}, req interface{}) *MockDataCoord_DropIndex_Call {
return &MockDataCoord_DropIndex_Call{Call: _e.mock.On("DropIndex", ctx, req)}
}
@ -398,8 +398,8 @@ type MockDataCoord_DropVirtualChannel_Call struct {
}
// DropVirtualChannel is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.DropVirtualChannelRequest
// - ctx context.Context
// - req *datapb.DropVirtualChannelRequest
func (_e *MockDataCoord_Expecter) DropVirtualChannel(ctx interface{}, req interface{}) *MockDataCoord_DropVirtualChannel_Call {
return &MockDataCoord_DropVirtualChannel_Call{Call: _e.mock.On("DropVirtualChannel", ctx, req)}
}
@ -453,8 +453,8 @@ type MockDataCoord_Flush_Call struct {
}
// Flush is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.FlushRequest
// - ctx context.Context
// - req *datapb.FlushRequest
func (_e *MockDataCoord_Expecter) Flush(ctx interface{}, req interface{}) *MockDataCoord_Flush_Call {
return &MockDataCoord_Flush_Call{Call: _e.mock.On("Flush", ctx, req)}
}
@ -508,8 +508,8 @@ type MockDataCoord_GcConfirm_Call struct {
}
// GcConfirm is a helper method to define mock.On call
// - ctx context.Context
// - request *datapb.GcConfirmRequest
// - ctx context.Context
// - request *datapb.GcConfirmRequest
func (_e *MockDataCoord_Expecter) GcConfirm(ctx interface{}, request interface{}) *MockDataCoord_GcConfirm_Call {
return &MockDataCoord_GcConfirm_Call{Call: _e.mock.On("GcConfirm", ctx, request)}
}
@ -563,8 +563,8 @@ type MockDataCoord_GetCollectionStatistics_Call struct {
}
// GetCollectionStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetCollectionStatisticsRequest
// - ctx context.Context
// - req *datapb.GetCollectionStatisticsRequest
func (_e *MockDataCoord_Expecter) GetCollectionStatistics(ctx interface{}, req interface{}) *MockDataCoord_GetCollectionStatistics_Call {
return &MockDataCoord_GetCollectionStatistics_Call{Call: _e.mock.On("GetCollectionStatistics", ctx, req)}
}
@ -618,8 +618,8 @@ type MockDataCoord_GetCompactionState_Call struct {
}
// GetCompactionState is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetCompactionStateRequest
// - ctx context.Context
// - req *milvuspb.GetCompactionStateRequest
func (_e *MockDataCoord_Expecter) GetCompactionState(ctx interface{}, req interface{}) *MockDataCoord_GetCompactionState_Call {
return &MockDataCoord_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", ctx, req)}
}
@ -673,8 +673,8 @@ type MockDataCoord_GetCompactionStateWithPlans_Call struct {
}
// GetCompactionStateWithPlans is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetCompactionPlansRequest
// - ctx context.Context
// - req *milvuspb.GetCompactionPlansRequest
func (_e *MockDataCoord_Expecter) GetCompactionStateWithPlans(ctx interface{}, req interface{}) *MockDataCoord_GetCompactionStateWithPlans_Call {
return &MockDataCoord_GetCompactionStateWithPlans_Call{Call: _e.mock.On("GetCompactionStateWithPlans", ctx, req)}
}
@ -728,7 +728,7 @@ type MockDataCoord_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockDataCoord_Expecter) GetComponentStates(ctx interface{}) *MockDataCoord_GetComponentStates_Call {
return &MockDataCoord_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", ctx)}
}
@ -782,8 +782,8 @@ type MockDataCoord_GetFlushAllState_Call struct {
}
// GetFlushAllState is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetFlushAllStateRequest
// - ctx context.Context
// - req *milvuspb.GetFlushAllStateRequest
func (_e *MockDataCoord_Expecter) GetFlushAllState(ctx interface{}, req interface{}) *MockDataCoord_GetFlushAllState_Call {
return &MockDataCoord_GetFlushAllState_Call{Call: _e.mock.On("GetFlushAllState", ctx, req)}
}
@ -806,15 +806,15 @@ func (_c *MockDataCoord_GetFlushAllState_Call) RunAndReturn(run func(context.Con
}
// GetFlushState provides a mock function with given fields: ctx, req
func (_m *MockDataCoord) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
func (_m *MockDataCoord) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
ret := _m.Called(ctx, req)
var r0 *milvuspb.GetFlushStateResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)); ok {
if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)); ok {
return rf(ctx, req)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetFlushStateRequest) *milvuspb.GetFlushStateResponse); ok {
if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetFlushStateRequest) *milvuspb.GetFlushStateResponse); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
@ -822,7 +822,7 @@ func (_m *MockDataCoord) GetFlushState(ctx context.Context, req *milvuspb.GetFlu
}
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetFlushStateRequest) error); ok {
if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetFlushStateRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
@ -837,15 +837,15 @@ type MockDataCoord_GetFlushState_Call struct {
}
// GetFlushState is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetFlushStateRequest
// - ctx context.Context
// - req *datapb.GetFlushStateRequest
func (_e *MockDataCoord_Expecter) GetFlushState(ctx interface{}, req interface{}) *MockDataCoord_GetFlushState_Call {
return &MockDataCoord_GetFlushState_Call{Call: _e.mock.On("GetFlushState", ctx, req)}
}
func (_c *MockDataCoord_GetFlushState_Call) Run(run func(ctx context.Context, req *milvuspb.GetFlushStateRequest)) *MockDataCoord_GetFlushState_Call {
func (_c *MockDataCoord_GetFlushState_Call) Run(run func(ctx context.Context, req *datapb.GetFlushStateRequest)) *MockDataCoord_GetFlushState_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*milvuspb.GetFlushStateRequest))
run(args[0].(context.Context), args[1].(*datapb.GetFlushStateRequest))
})
return _c
}
@ -855,7 +855,7 @@ func (_c *MockDataCoord_GetFlushState_Call) Return(_a0 *milvuspb.GetFlushStateRe
return _c
}
func (_c *MockDataCoord_GetFlushState_Call) RunAndReturn(run func(context.Context, *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)) *MockDataCoord_GetFlushState_Call {
func (_c *MockDataCoord_GetFlushState_Call) RunAndReturn(run func(context.Context, *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)) *MockDataCoord_GetFlushState_Call {
_c.Call.Return(run)
return _c
}
@ -892,8 +892,8 @@ type MockDataCoord_GetFlushedSegments_Call struct {
}
// GetFlushedSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetFlushedSegmentsRequest
// - ctx context.Context
// - req *datapb.GetFlushedSegmentsRequest
func (_e *MockDataCoord_Expecter) GetFlushedSegments(ctx interface{}, req interface{}) *MockDataCoord_GetFlushedSegments_Call {
return &MockDataCoord_GetFlushedSegments_Call{Call: _e.mock.On("GetFlushedSegments", ctx, req)}
}
@ -947,8 +947,8 @@ type MockDataCoord_GetIndexBuildProgress_Call struct {
}
// GetIndexBuildProgress is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexBuildProgressRequest
// - ctx context.Context
// - req *indexpb.GetIndexBuildProgressRequest
func (_e *MockDataCoord_Expecter) GetIndexBuildProgress(ctx interface{}, req interface{}) *MockDataCoord_GetIndexBuildProgress_Call {
return &MockDataCoord_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress", ctx, req)}
}
@ -1002,8 +1002,8 @@ type MockDataCoord_GetIndexInfos_Call struct {
}
// GetIndexInfos is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexInfoRequest
// - ctx context.Context
// - req *indexpb.GetIndexInfoRequest
func (_e *MockDataCoord_Expecter) GetIndexInfos(ctx interface{}, req interface{}) *MockDataCoord_GetIndexInfos_Call {
return &MockDataCoord_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos", ctx, req)}
}
@ -1057,8 +1057,8 @@ type MockDataCoord_GetIndexState_Call struct {
}
// GetIndexState is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexStateRequest
// - ctx context.Context
// - req *indexpb.GetIndexStateRequest
func (_e *MockDataCoord_Expecter) GetIndexState(ctx interface{}, req interface{}) *MockDataCoord_GetIndexState_Call {
return &MockDataCoord_GetIndexState_Call{Call: _e.mock.On("GetIndexState", ctx, req)}
}
@ -1112,8 +1112,8 @@ type MockDataCoord_GetIndexStatistics_Call struct {
}
// GetIndexStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetIndexStatisticsRequest
// - ctx context.Context
// - req *indexpb.GetIndexStatisticsRequest
func (_e *MockDataCoord_Expecter) GetIndexStatistics(ctx interface{}, req interface{}) *MockDataCoord_GetIndexStatistics_Call {
return &MockDataCoord_GetIndexStatistics_Call{Call: _e.mock.On("GetIndexStatistics", ctx, req)}
}
@ -1167,8 +1167,8 @@ type MockDataCoord_GetInsertBinlogPaths_Call struct {
}
// GetInsertBinlogPaths is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetInsertBinlogPathsRequest
// - ctx context.Context
// - req *datapb.GetInsertBinlogPathsRequest
func (_e *MockDataCoord_Expecter) GetInsertBinlogPaths(ctx interface{}, req interface{}) *MockDataCoord_GetInsertBinlogPaths_Call {
return &MockDataCoord_GetInsertBinlogPaths_Call{Call: _e.mock.On("GetInsertBinlogPaths", ctx, req)}
}
@ -1222,8 +1222,8 @@ type MockDataCoord_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
func (_e *MockDataCoord_Expecter) GetMetrics(ctx interface{}, req interface{}) *MockDataCoord_GetMetrics_Call {
return &MockDataCoord_GetMetrics_Call{Call: _e.mock.On("GetMetrics", ctx, req)}
}
@ -1277,8 +1277,8 @@ type MockDataCoord_GetPartitionStatistics_Call struct {
}
// GetPartitionStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetPartitionStatisticsRequest
// - ctx context.Context
// - req *datapb.GetPartitionStatisticsRequest
func (_e *MockDataCoord_Expecter) GetPartitionStatistics(ctx interface{}, req interface{}) *MockDataCoord_GetPartitionStatistics_Call {
return &MockDataCoord_GetPartitionStatistics_Call{Call: _e.mock.On("GetPartitionStatistics", ctx, req)}
}
@ -1332,8 +1332,8 @@ type MockDataCoord_GetRecoveryInfo_Call struct {
}
// GetRecoveryInfo is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequest
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequest
func (_e *MockDataCoord_Expecter) GetRecoveryInfo(ctx interface{}, req interface{}) *MockDataCoord_GetRecoveryInfo_Call {
return &MockDataCoord_GetRecoveryInfo_Call{Call: _e.mock.On("GetRecoveryInfo", ctx, req)}
}
@ -1387,8 +1387,8 @@ type MockDataCoord_GetRecoveryInfoV2_Call struct {
}
// GetRecoveryInfoV2 is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequestV2
// - ctx context.Context
// - req *datapb.GetRecoveryInfoRequestV2
func (_e *MockDataCoord_Expecter) GetRecoveryInfoV2(ctx interface{}, req interface{}) *MockDataCoord_GetRecoveryInfoV2_Call {
return &MockDataCoord_GetRecoveryInfoV2_Call{Call: _e.mock.On("GetRecoveryInfoV2", ctx, req)}
}
@ -1442,8 +1442,8 @@ type MockDataCoord_GetSegmentIndexState_Call struct {
}
// GetSegmentIndexState is a helper method to define mock.On call
// - ctx context.Context
// - req *indexpb.GetSegmentIndexStateRequest
// - ctx context.Context
// - req *indexpb.GetSegmentIndexStateRequest
func (_e *MockDataCoord_Expecter) GetSegmentIndexState(ctx interface{}, req interface{}) *MockDataCoord_GetSegmentIndexState_Call {
return &MockDataCoord_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState", ctx, req)}
}
@ -1497,8 +1497,8 @@ type MockDataCoord_GetSegmentInfo_Call struct {
}
// GetSegmentInfo is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetSegmentInfoRequest
// - ctx context.Context
// - req *datapb.GetSegmentInfoRequest
func (_e *MockDataCoord_Expecter) GetSegmentInfo(ctx interface{}, req interface{}) *MockDataCoord_GetSegmentInfo_Call {
return &MockDataCoord_GetSegmentInfo_Call{Call: _e.mock.On("GetSegmentInfo", ctx, req)}
}
@ -1552,7 +1552,7 @@ type MockDataCoord_GetSegmentInfoChannel_Call struct {
}
// GetSegmentInfoChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockDataCoord_Expecter) GetSegmentInfoChannel(ctx interface{}) *MockDataCoord_GetSegmentInfoChannel_Call {
return &MockDataCoord_GetSegmentInfoChannel_Call{Call: _e.mock.On("GetSegmentInfoChannel", ctx)}
}
@ -1606,8 +1606,8 @@ type MockDataCoord_GetSegmentStates_Call struct {
}
// GetSegmentStates is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetSegmentStatesRequest
// - ctx context.Context
// - req *datapb.GetSegmentStatesRequest
func (_e *MockDataCoord_Expecter) GetSegmentStates(ctx interface{}, req interface{}) *MockDataCoord_GetSegmentStates_Call {
return &MockDataCoord_GetSegmentStates_Call{Call: _e.mock.On("GetSegmentStates", ctx, req)}
}
@ -1661,8 +1661,8 @@ type MockDataCoord_GetSegmentsByStates_Call struct {
}
// GetSegmentsByStates is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.GetSegmentsByStatesRequest
// - ctx context.Context
// - req *datapb.GetSegmentsByStatesRequest
func (_e *MockDataCoord_Expecter) GetSegmentsByStates(ctx interface{}, req interface{}) *MockDataCoord_GetSegmentsByStates_Call {
return &MockDataCoord_GetSegmentsByStates_Call{Call: _e.mock.On("GetSegmentsByStates", ctx, req)}
}
@ -1716,7 +1716,7 @@ type MockDataCoord_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockDataCoord_Expecter) GetStatisticsChannel(ctx interface{}) *MockDataCoord_GetStatisticsChannel_Call {
return &MockDataCoord_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", ctx)}
}
@ -1770,7 +1770,7 @@ type MockDataCoord_GetTimeTickChannel_Call struct {
}
// GetTimeTickChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockDataCoord_Expecter) GetTimeTickChannel(ctx interface{}) *MockDataCoord_GetTimeTickChannel_Call {
return &MockDataCoord_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel", ctx)}
}
@ -1824,8 +1824,8 @@ type MockDataCoord_Import_Call struct {
}
// Import is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ImportTaskRequest
// - ctx context.Context
// - req *datapb.ImportTaskRequest
func (_e *MockDataCoord_Expecter) Import(ctx interface{}, req interface{}) *MockDataCoord_Import_Call {
return &MockDataCoord_Import_Call{Call: _e.mock.On("Import", ctx, req)}
}
@ -1920,8 +1920,8 @@ type MockDataCoord_ManualCompaction_Call struct {
}
// ManualCompaction is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.ManualCompactionRequest
// - ctx context.Context
// - req *milvuspb.ManualCompactionRequest
func (_e *MockDataCoord_Expecter) ManualCompaction(ctx interface{}, req interface{}) *MockDataCoord_ManualCompaction_Call {
return &MockDataCoord_ManualCompaction_Call{Call: _e.mock.On("ManualCompaction", ctx, req)}
}
@ -1975,8 +1975,8 @@ type MockDataCoord_MarkSegmentsDropped_Call struct {
}
// MarkSegmentsDropped is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.MarkSegmentsDroppedRequest
// - ctx context.Context
// - req *datapb.MarkSegmentsDroppedRequest
func (_e *MockDataCoord_Expecter) MarkSegmentsDropped(ctx interface{}, req interface{}) *MockDataCoord_MarkSegmentsDropped_Call {
return &MockDataCoord_MarkSegmentsDropped_Call{Call: _e.mock.On("MarkSegmentsDropped", ctx, req)}
}
@ -2071,8 +2071,8 @@ type MockDataCoord_ReportDataNodeTtMsgs_Call struct {
}
// ReportDataNodeTtMsgs is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ReportDataNodeTtMsgsRequest
// - ctx context.Context
// - req *datapb.ReportDataNodeTtMsgsRequest
func (_e *MockDataCoord_Expecter) ReportDataNodeTtMsgs(ctx interface{}, req interface{}) *MockDataCoord_ReportDataNodeTtMsgs_Call {
return &MockDataCoord_ReportDataNodeTtMsgs_Call{Call: _e.mock.On("ReportDataNodeTtMsgs", ctx, req)}
}
@ -2126,8 +2126,8 @@ type MockDataCoord_SaveBinlogPaths_Call struct {
}
// SaveBinlogPaths is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SaveBinlogPathsRequest
// - ctx context.Context
// - req *datapb.SaveBinlogPathsRequest
func (_e *MockDataCoord_Expecter) SaveBinlogPaths(ctx interface{}, req interface{}) *MockDataCoord_SaveBinlogPaths_Call {
return &MockDataCoord_SaveBinlogPaths_Call{Call: _e.mock.On("SaveBinlogPaths", ctx, req)}
}
@ -2181,8 +2181,8 @@ type MockDataCoord_SaveImportSegment_Call struct {
}
// SaveImportSegment is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SaveImportSegmentRequest
// - ctx context.Context
// - req *datapb.SaveImportSegmentRequest
func (_e *MockDataCoord_Expecter) SaveImportSegment(ctx interface{}, req interface{}) *MockDataCoord_SaveImportSegment_Call {
return &MockDataCoord_SaveImportSegment_Call{Call: _e.mock.On("SaveImportSegment", ctx, req)}
}
@ -2215,7 +2215,7 @@ type MockDataCoord_SetAddress_Call struct {
}
// SetAddress is a helper method to define mock.On call
// - address string
// - address string
func (_e *MockDataCoord_Expecter) SetAddress(address interface{}) *MockDataCoord_SetAddress_Call {
return &MockDataCoord_SetAddress_Call{Call: _e.mock.On("SetAddress", address)}
}
@ -2248,7 +2248,7 @@ type MockDataCoord_SetDataNodeCreator_Call struct {
}
// SetDataNodeCreator is a helper method to define mock.On call
// - _a0 func(context.Context , string , int64)(types.DataNode , error)
// - _a0 func(context.Context , string , int64)(types.DataNode , error)
func (_e *MockDataCoord_Expecter) SetDataNodeCreator(_a0 interface{}) *MockDataCoord_SetDataNodeCreator_Call {
return &MockDataCoord_SetDataNodeCreator_Call{Call: _e.mock.On("SetDataNodeCreator", _a0)}
}
@ -2281,7 +2281,7 @@ type MockDataCoord_SetEtcdClient_Call struct {
}
// SetEtcdClient is a helper method to define mock.On call
// - etcdClient *clientv3.Client
// - etcdClient *clientv3.Client
func (_e *MockDataCoord_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataCoord_SetEtcdClient_Call {
return &MockDataCoord_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)}
}
@ -2314,7 +2314,7 @@ type MockDataCoord_SetIndexNodeCreator_Call struct {
}
// SetIndexNodeCreator is a helper method to define mock.On call
// - _a0 func(context.Context , string , int64)(types.IndexNode , error)
// - _a0 func(context.Context , string , int64)(types.IndexNode , error)
func (_e *MockDataCoord_Expecter) SetIndexNodeCreator(_a0 interface{}) *MockDataCoord_SetIndexNodeCreator_Call {
return &MockDataCoord_SetIndexNodeCreator_Call{Call: _e.mock.On("SetIndexNodeCreator", _a0)}
}
@ -2347,7 +2347,7 @@ type MockDataCoord_SetRootCoord_Call struct {
}
// SetRootCoord is a helper method to define mock.On call
// - rootCoord types.RootCoord
// - rootCoord types.RootCoord
func (_e *MockDataCoord_Expecter) SetRootCoord(rootCoord interface{}) *MockDataCoord_SetRootCoord_Call {
return &MockDataCoord_SetRootCoord_Call{Call: _e.mock.On("SetRootCoord", rootCoord)}
}
@ -2401,8 +2401,8 @@ type MockDataCoord_SetSegmentState_Call struct {
}
// SetSegmentState is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SetSegmentStateRequest
// - ctx context.Context
// - req *datapb.SetSegmentStateRequest
func (_e *MockDataCoord_Expecter) SetSegmentState(ctx interface{}, req interface{}) *MockDataCoord_SetSegmentState_Call {
return &MockDataCoord_SetSegmentState_Call{Call: _e.mock.On("SetSegmentState", ctx, req)}
}
@ -2489,8 +2489,8 @@ type MockDataCoord_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
func (_e *MockDataCoord_Expecter) ShowConfigurations(ctx interface{}, req interface{}) *MockDataCoord_ShowConfigurations_Call {
return &MockDataCoord_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", ctx, req)}
}
@ -2626,8 +2626,8 @@ type MockDataCoord_UnsetIsImportingState_Call struct {
}
// UnsetIsImportingState is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.UnsetIsImportingStateRequest
// - ctx context.Context
// - req *datapb.UnsetIsImportingStateRequest
func (_e *MockDataCoord_Expecter) UnsetIsImportingState(ctx interface{}, req interface{}) *MockDataCoord_UnsetIsImportingState_Call {
return &MockDataCoord_UnsetIsImportingState_Call{Call: _e.mock.On("UnsetIsImportingState", ctx, req)}
}
@ -2681,8 +2681,8 @@ type MockDataCoord_UpdateChannelCheckpoint_Call struct {
}
// UpdateChannelCheckpoint is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.UpdateChannelCheckpointRequest
// - ctx context.Context
// - req *datapb.UpdateChannelCheckpointRequest
func (_e *MockDataCoord_Expecter) UpdateChannelCheckpoint(ctx interface{}, req interface{}) *MockDataCoord_UpdateChannelCheckpoint_Call {
return &MockDataCoord_UpdateChannelCheckpoint_Call{Call: _e.mock.On("UpdateChannelCheckpoint", ctx, req)}
}
@ -2736,8 +2736,8 @@ type MockDataCoord_UpdateSegmentStatistics_Call struct {
}
// UpdateSegmentStatistics is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.UpdateSegmentStatisticsRequest
// - ctx context.Context
// - req *datapb.UpdateSegmentStatisticsRequest
func (_e *MockDataCoord_Expecter) UpdateSegmentStatistics(ctx interface{}, req interface{}) *MockDataCoord_UpdateSegmentStatistics_Call {
return &MockDataCoord_UpdateSegmentStatistics_Call{Call: _e.mock.On("UpdateSegmentStatistics", ctx, req)}
}
@ -2791,8 +2791,8 @@ type MockDataCoord_WatchChannels_Call struct {
}
// WatchChannels is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.WatchChannelsRequest
// - ctx context.Context
// - req *datapb.WatchChannelsRequest
func (_e *MockDataCoord_Expecter) WatchChannels(ctx interface{}, req interface{}) *MockDataCoord_WatchChannels_Call {
return &MockDataCoord_WatchChannels_Call{Call: _e.mock.On("WatchChannels", ctx, req)}
}

View File

@ -64,8 +64,8 @@ type MockDataNode_AddImportSegment_Call struct {
}
// AddImportSegment is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.AddImportSegmentRequest
// - ctx context.Context
// - req *datapb.AddImportSegmentRequest
func (_e *MockDataNode_Expecter) AddImportSegment(ctx interface{}, req interface{}) *MockDataNode_AddImportSegment_Call {
return &MockDataNode_AddImportSegment_Call{Call: _e.mock.On("AddImportSegment", ctx, req)}
}
@ -119,8 +119,8 @@ type MockDataNode_Compaction_Call struct {
}
// Compaction is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.CompactionPlan
// - ctx context.Context
// - req *datapb.CompactionPlan
func (_e *MockDataNode_Expecter) Compaction(ctx interface{}, req interface{}) *MockDataNode_Compaction_Call {
return &MockDataNode_Compaction_Call{Call: _e.mock.On("Compaction", ctx, req)}
}
@ -142,6 +142,61 @@ func (_c *MockDataNode_Compaction_Call) RunAndReturn(run func(context.Context, *
return _c
}
// FlushChannels provides a mock function with given fields: ctx, req
func (_m *MockDataNode) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
var r0 *commonpb.Status
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.FlushChannelsRequest) (*commonpb.Status, error)); ok {
return rf(ctx, req)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.FlushChannelsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.FlushChannelsRequest) error); ok {
r1 = rf(ctx, req)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockDataNode_FlushChannels_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'FlushChannels'
type MockDataNode_FlushChannels_Call struct {
*mock.Call
}
// FlushChannels is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.FlushChannelsRequest
func (_e *MockDataNode_Expecter) FlushChannels(ctx interface{}, req interface{}) *MockDataNode_FlushChannels_Call {
return &MockDataNode_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, req)}
}
func (_c *MockDataNode_FlushChannels_Call) Run(run func(ctx context.Context, req *datapb.FlushChannelsRequest)) *MockDataNode_FlushChannels_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.FlushChannelsRequest))
})
return _c
}
func (_c *MockDataNode_FlushChannels_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNode_FlushChannels_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataNode_FlushChannels_Call) RunAndReturn(run func(context.Context, *datapb.FlushChannelsRequest) (*commonpb.Status, error)) *MockDataNode_FlushChannels_Call {
_c.Call.Return(run)
return _c
}
// FlushSegments provides a mock function with given fields: ctx, req
func (_m *MockDataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
@ -174,8 +229,8 @@ type MockDataNode_FlushSegments_Call struct {
}
// FlushSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.FlushSegmentsRequest
// - ctx context.Context
// - req *datapb.FlushSegmentsRequest
func (_e *MockDataNode_Expecter) FlushSegments(ctx interface{}, req interface{}) *MockDataNode_FlushSegments_Call {
return &MockDataNode_FlushSegments_Call{Call: _e.mock.On("FlushSegments", ctx, req)}
}
@ -270,8 +325,8 @@ type MockDataNode_GetCompactionState_Call struct {
}
// GetCompactionState is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.CompactionStateRequest
// - ctx context.Context
// - req *datapb.CompactionStateRequest
func (_e *MockDataNode_Expecter) GetCompactionState(ctx interface{}, req interface{}) *MockDataNode_GetCompactionState_Call {
return &MockDataNode_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", ctx, req)}
}
@ -325,7 +380,7 @@ type MockDataNode_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockDataNode_Expecter) GetComponentStates(ctx interface{}) *MockDataNode_GetComponentStates_Call {
return &MockDataNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", ctx)}
}
@ -379,8 +434,8 @@ type MockDataNode_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
func (_e *MockDataNode_Expecter) GetMetrics(ctx interface{}, req interface{}) *MockDataNode_GetMetrics_Call {
return &MockDataNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", ctx, req)}
}
@ -475,7 +530,7 @@ type MockDataNode_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockDataNode_Expecter) GetStatisticsChannel(ctx interface{}) *MockDataNode_GetStatisticsChannel_Call {
return &MockDataNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", ctx)}
}
@ -529,8 +584,8 @@ type MockDataNode_Import_Call struct {
}
// Import is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ImportTaskRequest
// - ctx context.Context
// - req *datapb.ImportTaskRequest
func (_e *MockDataNode_Expecter) Import(ctx interface{}, req interface{}) *MockDataNode_Import_Call {
return &MockDataNode_Import_Call{Call: _e.mock.On("Import", ctx, req)}
}
@ -666,8 +721,8 @@ type MockDataNode_ResendSegmentStats_Call struct {
}
// ResendSegmentStats is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ResendSegmentStatsRequest
// - ctx context.Context
// - req *datapb.ResendSegmentStatsRequest
func (_e *MockDataNode_Expecter) ResendSegmentStats(ctx interface{}, req interface{}) *MockDataNode_ResendSegmentStats_Call {
return &MockDataNode_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", ctx, req)}
}
@ -700,7 +755,7 @@ type MockDataNode_SetAddress_Call struct {
}
// SetAddress is a helper method to define mock.On call
// - address string
// - address string
func (_e *MockDataNode_Expecter) SetAddress(address interface{}) *MockDataNode_SetAddress_Call {
return &MockDataNode_SetAddress_Call{Call: _e.mock.On("SetAddress", address)}
}
@ -742,7 +797,7 @@ type MockDataNode_SetDataCoord_Call struct {
}
// SetDataCoord is a helper method to define mock.On call
// - dataCoord types.DataCoord
// - dataCoord types.DataCoord
func (_e *MockDataNode_Expecter) SetDataCoord(dataCoord interface{}) *MockDataNode_SetDataCoord_Call {
return &MockDataNode_SetDataCoord_Call{Call: _e.mock.On("SetDataCoord", dataCoord)}
}
@ -775,7 +830,7 @@ type MockDataNode_SetEtcdClient_Call struct {
}
// SetEtcdClient is a helper method to define mock.On call
// - etcdClient *clientv3.Client
// - etcdClient *clientv3.Client
func (_e *MockDataNode_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataNode_SetEtcdClient_Call {
return &MockDataNode_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)}
}
@ -817,7 +872,7 @@ type MockDataNode_SetRootCoord_Call struct {
}
// SetRootCoord is a helper method to define mock.On call
// - rootCoord types.RootCoord
// - rootCoord types.RootCoord
func (_e *MockDataNode_Expecter) SetRootCoord(rootCoord interface{}) *MockDataNode_SetRootCoord_Call {
return &MockDataNode_SetRootCoord_Call{Call: _e.mock.On("SetRootCoord", rootCoord)}
}
@ -871,8 +926,8 @@ type MockDataNode_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
func (_e *MockDataNode_Expecter) ShowConfigurations(ctx interface{}, req interface{}) *MockDataNode_ShowConfigurations_Call {
return &MockDataNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", ctx, req)}
}
@ -1008,8 +1063,8 @@ type MockDataNode_SyncSegments_Call struct {
}
// SyncSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SyncSegmentsRequest
// - ctx context.Context
// - req *datapb.SyncSegmentsRequest
func (_e *MockDataNode_Expecter) SyncSegments(ctx interface{}, req interface{}) *MockDataNode_SyncSegments_Call {
return &MockDataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", ctx, req)}
}
@ -1042,7 +1097,7 @@ type MockDataNode_UpdateStateCode_Call struct {
}
// UpdateStateCode is a helper method to define mock.On call
// - stateCode commonpb.StateCode
// - stateCode commonpb.StateCode
func (_e *MockDataNode_Expecter) UpdateStateCode(stateCode interface{}) *MockDataNode_UpdateStateCode_Call {
return &MockDataNode_UpdateStateCode_Call{Call: _e.mock.On("UpdateStateCode", stateCode)}
}
@ -1096,8 +1151,8 @@ type MockDataNode_WatchDmChannels_Call struct {
}
// WatchDmChannels is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.WatchDmChannelsRequest
// - ctx context.Context
// - req *datapb.WatchDmChannelsRequest
func (_e *MockDataNode_Expecter) WatchDmChannels(ctx interface{}, req interface{}) *MockDataNode_WatchDmChannels_Call {
return &MockDataNode_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", ctx, req)}
}

View File

@ -54,7 +54,7 @@ service DataCoord {
rpc GetCompactionStateWithPlans(milvus.GetCompactionPlansRequest) returns (milvus.GetCompactionPlansResponse) {}
rpc WatchChannels(WatchChannelsRequest) returns (WatchChannelsResponse) {}
rpc GetFlushState(milvus.GetFlushStateRequest) returns (milvus.GetFlushStateResponse) {}
rpc GetFlushState(GetFlushStateRequest) returns (milvus.GetFlushStateResponse) {}
rpc DropVirtualChannel(DropVirtualChannelRequest) returns (DropVirtualChannelResponse) {}
rpc SetSegmentState(SetSegmentStateRequest) returns (SetSegmentStateResponse) {}
@ -108,6 +108,8 @@ service DataNode {
rpc ResendSegmentStats(ResendSegmentStatsRequest) returns(ResendSegmentStatsResponse) {}
rpc AddImportSegment(AddImportSegmentRequest) returns(AddImportSegmentResponse) {}
rpc FlushChannels(FlushChannelsRequest) returns(common.Status) {}
}
message FlushRequest {
@ -125,6 +127,13 @@ message FlushResponse {
repeated int64 segmentIDs = 4; // newly sealed segments
repeated int64 flushSegmentIDs = 5; // old flushed segment
int64 timeOfSeal = 6;
uint64 flush_ts = 7;
}
message FlushChannelsRequest {
common.MsgBase base = 1;
uint64 flush_ts = 2;
repeated string channels = 3;
}
message SegmentIDRequest {
@ -677,3 +686,11 @@ message ReportDataNodeTtMsgsRequest {
common.MsgBase base = 1;
repeated msg.DataNodeTtMsg msgs = 2; // -1 means whole collection.
}
message GetFlushStateRequest {
repeated int64 segmentIDs = 1;
uint64 flush_ts = 2;
string db_name = 3;
string collection_name = 4;
int64 collectionID = 5;
}

File diff suppressed because it is too large Load Diff

View File

@ -254,7 +254,7 @@ func (coord *DataCoordMock) WatchChannels(ctx context.Context, req *datapb.Watch
return &datapb.WatchChannelsResponse{}, nil
}
func (coord *DataCoordMock) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
func (coord *DataCoordMock) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return &milvuspb.GetFlushStateResponse{}, nil
}

View File

@ -3600,7 +3600,7 @@ func (node *Proxy) GetCompactionStateWithPlans(ctx context.Context, req *milvusp
return resp, err
}
// GetFlushState gets the flush state of multiple segments
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
func (node *Proxy) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetFlushState")
defer sp.End()
@ -3617,11 +3617,28 @@ func (node *Proxy) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStat
return resp, nil
}
resp, err = node.dataCoord.GetFlushState(ctx, req)
if err = validateCollectionName(req.GetCollectionName()); err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
stateReq := &datapb.GetFlushStateRequest{
SegmentIDs: req.GetSegmentIDs(),
FlushTs: req.GetFlushTs(),
}
stateReq.CollectionID = collectionID
resp, err = node.dataCoord.GetFlushState(ctx, stateReq)
if err != nil {
log.Warn("failed to get flush state response",
zap.Error(err))
return nil, err
resp.Status = merr.Status(err)
return resp, nil
}
log.Debug("received get flush state response",
zap.Any("response", resp))

View File

@ -4041,10 +4041,24 @@ func Test_GetCompactionStateWithPlans(t *testing.T) {
func Test_GetFlushState(t *testing.T) {
t.Run("normal test", func(t *testing.T) {
originCache := globalMetaCache
m := NewMockCache(t)
m.On("GetCollectionID",
mock.Anything,
mock.AnythingOfType("string"),
mock.AnythingOfType("string"),
).Return(UniqueID(1), nil)
globalMetaCache = m
defer func() {
globalMetaCache = originCache
}()
datacoord := &DataCoordMock{}
proxy := &Proxy{dataCoord: datacoord}
proxy.stateCode.Store(commonpb.StateCode_Healthy)
resp, err := proxy.GetFlushState(context.TODO(), nil)
resp, err := proxy.GetFlushState(context.TODO(), &milvuspb.GetFlushStateRequest{
CollectionName: "coll",
})
assert.EqualValues(t, &milvuspb.GetFlushStateResponse{}, resp)
assert.NoError(t, err)
})

View File

@ -1286,6 +1286,7 @@ func (ft *flushTask) Execute(ctx context.Context) error {
coll2Segments := make(map[string]*schemapb.LongArray)
flushColl2Segments := make(map[string]*schemapb.LongArray)
coll2SealTimes := make(map[string]int64)
coll2FlushTs := make(map[string]Timestamp)
for _, collName := range ft.CollectionNames {
collID, err := globalMetaCache.GetCollectionID(ctx, ft.GetDbName(), collName)
if err != nil {
@ -1309,6 +1310,7 @@ func (ft *flushTask) Execute(ctx context.Context) error {
coll2Segments[collName] = &schemapb.LongArray{Data: resp.GetSegmentIDs()}
flushColl2Segments[collName] = &schemapb.LongArray{Data: resp.GetFlushSegmentIDs()}
coll2SealTimes[collName] = resp.GetTimeOfSeal()
coll2FlushTs[collName] = resp.GetFlushTs()
}
ft.result = &milvuspb.FlushResponse{
Status: merr.Status(nil),
@ -1316,6 +1318,7 @@ func (ft *flushTask) Execute(ctx context.Context) error {
CollSegIDs: coll2Segments,
FlushCollSegIDs: flushColl2Segments,
CollSealTimes: coll2SealTimes,
CollFlushTs: coll2FlushTs,
}
return nil
}

View File

@ -102,6 +102,9 @@ type DataNode interface {
// AddImportSegment puts the given import segment to current DataNode's flow graph.
AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)
// FlushChannels notifies DataNode to sync all the segments belongs to the target channels.
FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error)
}
// DataNodeComponent is used by grpc server of DataNode
@ -147,7 +150,8 @@ type DataCoord interface {
Component
TimeTickProvider
// Flush notifies DataCoord to flush all current growing segments of specified Collection
// Flush notifies DataCoord to flush all current growing segments of specified Collection,
// and flush all the buffers in DataNode and MsgStream into storage.
// ctx is the context to control request deadline and cancellation
// req contains the request params, which are database name(not used for now) and collection id
//
@ -302,8 +306,8 @@ type DataCoord interface {
// WatchChannels notifies DataCoord to watch vchannels of a collection
WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest) (*datapb.WatchChannelsResponse, error)
// GetFlushState gets the flush state of multiple segments
GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)
// GetFlushAllState checks if all DML messages before `FlushAllTs` have been flushed.
GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error)
// SetSegmentState updates a segment's state explicitly.
@ -1360,7 +1364,7 @@ type ProxyComponent interface {
GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error)
ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error)
GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error)
// GetFlushState gets the flush state of multiple segments
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error)
// GetFlushAllState checks if all DML messages before `FlushAllTs` have been flushed.
GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error)

View File

@ -112,7 +112,7 @@ func (m *DataCoordClient) GetCompactionStateWithPlans(ctx context.Context, req *
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) {
func (m *DataCoordClient) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) {
return &milvuspb.GetFlushStateResponse{}, m.Err
}

View File

@ -136,7 +136,7 @@ func (m *GrpcDataCoordClient) WatchChannels(ctx context.Context, req *datapb.Wat
return &datapb.WatchChannelsResponse{}, m.Err
}
func (m *GrpcDataCoordClient) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) {
func (m *GrpcDataCoordClient) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) {
return &milvuspb.GetFlushStateResponse{}, m.Err
}

View File

@ -80,3 +80,7 @@ func (m *GrpcDataNodeClient) AddImportSegment(ctx context.Context, in *datapb.Ad
func (m *GrpcDataNodeClient) SyncSegments(ctx context.Context, in *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcDataNodeClient) FlushChannels(ctx context.Context, in *datapb.FlushChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}

View File

@ -152,12 +152,14 @@ func (s *TestGetVectorSuite) run() {
ids := segmentIDs.GetData()
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collection]
s.Require().True(has)
segments, err := s.Cluster.MetaWatcher.ShowSegments()
s.Require().NoError(err)
s.Require().NotEmpty(segments)
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, s.dbName, collection)
// create index
_, err = s.Cluster.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{

View File

@ -94,7 +94,9 @@ func (s *HelloMilvusSuite) TestHelloMilvus() {
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
segments, err := c.MetaWatcher.ShowSegments()
@ -103,7 +105,7 @@ func (s *HelloMilvusSuite) TestHelloMilvus() {
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{

View File

@ -68,9 +68,11 @@ func (s *GetIndexStatisticsSuite) TestGetIndexStatistics() {
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.Equal(true, has)
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
indexName := "_default"

View File

@ -24,7 +24,6 @@ import (
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/tests/integration"
"github.com/stretchr/testify/suite"
@ -733,7 +732,10 @@ func (s *JSONExprSuite) insertFlushIndexLoad(ctx context.Context, dbName, collec
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
segments, err := s.Cluster.MetaWatcher.ShowSegments()
s.NoError(err)
@ -741,28 +743,7 @@ func (s *JSONExprSuite) insertFlushIndexLoad(ctx context.Context, dbName, collec
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
if has && len(ids) > 0 {
flushed := func() bool {
resp, err := s.Cluster.Proxy.GetFlushState(ctx, &milvuspb.GetFlushStateRequest{
SegmentIDs: ids,
})
if err != nil {
//panic(errors.New("GetFlushState failed"))
return false
}
return resp.GetFlushed()
}
for !flushed() {
// respect context deadline/cancel
select {
case <-ctx.Done():
panic(errors.New("deadline exceeded"))
default:
}
time.Sleep(500 * time.Millisecond)
}
}
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := s.Cluster.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{

View File

@ -22,7 +22,6 @@ import (
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/suite"
"go.uber.org/zap"
@ -226,9 +225,6 @@ func (s *MetaWatcherSuite) TestShowReplicas() {
CollectionNames: []string{collectionName},
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
@ -236,28 +232,13 @@ func (s *MetaWatcherSuite) TestShowReplicas() {
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
if has && len(ids) > 0 {
flushed := func() bool {
resp, err := c.Proxy.GetFlushState(ctx, &milvuspb.GetFlushStateRequest{
SegmentIDs: ids,
})
if err != nil {
//panic(errors.New("GetFlushState failed"))
return false
}
return resp.GetFlushed()
}
for !flushed() {
// respect context deadline/cancel
select {
case <-ctx.Done():
panic(errors.New("deadline exceeded"))
default:
}
time.Sleep(500 * time.Millisecond)
}
}
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{

View File

@ -92,9 +92,11 @@ func (s *RangeSearchSuite) TestRangeSearchIP() {
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
s.True(has)
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
@ -102,7 +104,7 @@ func (s *RangeSearchSuite) TestRangeSearchIP() {
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
@ -240,9 +242,11 @@ func (s *RangeSearchSuite) TestRangeSearchL2() {
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
s.True(has)
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
@ -250,7 +254,7 @@ func (s *RangeSearchSuite) TestRangeSearchL2() {
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{

View File

@ -118,11 +118,13 @@ func (s *RefreshConfigSuite) TestRefreshDefaultIndexName() {
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
s.True(has)
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
_, err = c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
CollectionName: collectionName,

View File

@ -92,9 +92,11 @@ func (s *UpsertSuite) TestUpsert() {
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
s.True(has)
ids := segmentIDs.GetData()
s.NotEmpty(segmentIDs)
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
@ -102,7 +104,7 @@ func (s *UpsertSuite) TestUpsert() {
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
s.WaitForFlush(ctx, ids)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{

View File

@ -26,10 +26,13 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
)
func (s *MiniClusterSuite) WaitForFlush(ctx context.Context, segIDs []int64) {
func (s *MiniClusterSuite) WaitForFlush(ctx context.Context, segIDs []int64, flushTs uint64, dbName, collectionName string) {
flushed := func() bool {
resp, err := s.Cluster.Proxy.GetFlushState(ctx, &milvuspb.GetFlushStateRequest{
SegmentIDs: segIDs,
SegmentIDs: segIDs,
FlushTs: flushTs,
DbName: dbName,
CollectionName: collectionName,
})
if err != nil {
return false

View File

@ -12,7 +12,7 @@ allure-pytest==2.7.0
pytest-print==0.2.1
pytest-level==0.1.1
pytest-xdist==2.5.0
pymilvus==2.3.0b0.post1.dev127
pymilvus==2.3.0.post1.dev4
pytest-rerunfailures==9.1.1
git+https://github.com/Projectplace/pytest-tags
ndg-httpsclient