mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 17:48:29 +08:00
enhance: remove redundant resource key watch operation, just keep consistency of wal (#40235)
issue: #38399 related PR: #39522 - Just implement exclusive broadcaster between broadcast message with same resource key to keep same order in different wal. - After simplify the broadcast model, original watch-based broadcast is too complicated and redundant, remove it. --------- Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
679a145e8b
commit
f47ab31f23
@ -1445,7 +1445,6 @@ func TestImportV2(t *testing.T) {
|
||||
wal := mock_streaming.NewMockWALAccesser(t)
|
||||
b := mock_streaming.NewMockBroadcast(t)
|
||||
wal.EXPECT().Broadcast().Return(b).Maybe()
|
||||
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
streaming.SetWALForTest(wal)
|
||||
defer streaming.RecoverWALForTest()
|
||||
|
||||
|
||||
@ -34,19 +34,3 @@ func (b broadcast) Ack(ctx context.Context, req types.BroadcastAckRequest) error
|
||||
|
||||
return b.streamingCoordClient.Broadcast().Ack(ctx, req)
|
||||
}
|
||||
|
||||
func (b broadcast) BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error {
|
||||
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
|
||||
return ErrWALAccesserClosed
|
||||
}
|
||||
defer b.lifetime.Done()
|
||||
return b.streamingCoordClient.Broadcast().BlockUntilEvent(ctx, message.NewResourceKeyAckOneBroadcastEvent(rk))
|
||||
}
|
||||
|
||||
func (b broadcast) BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error {
|
||||
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
|
||||
return ErrWALAccesserClosed
|
||||
}
|
||||
defer b.lifetime.Done()
|
||||
return b.streamingCoordClient.Broadcast().BlockUntilEvent(ctx, message.NewResourceKeyAckAllBroadcastEvent(rk))
|
||||
}
|
||||
|
||||
@ -122,20 +122,14 @@ type WALAccesser interface {
|
||||
type Broadcast interface {
|
||||
// Append of Broadcast sends a broadcast message to all target vchannels.
|
||||
// Guarantees the atomicity written of the messages and eventual consistency.
|
||||
// The resource-key bound at the message will be held until the message is acked at consumer.
|
||||
// Once the resource-key is held, the append operation will be rejected.
|
||||
// Use resource-key to make a sequential operation at same resource-key.
|
||||
// The resource-key bound at the message will be held as a mutex until the message is broadcasted to all vchannels,
|
||||
// so the other append operation with the same resource-key will be searialized with a deterministic order on every vchannel.
|
||||
// The Append operation will be blocked until the message is consumed and acknowledged by the flusher at streamingnode.
|
||||
Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)
|
||||
|
||||
// Ack acknowledges a broadcast message at the specified vchannel.
|
||||
// It must be called after the message is comsumed by the unique-consumer.
|
||||
Ack(ctx context.Context, req types.BroadcastAckRequest) error
|
||||
|
||||
// BlockUntilResourceKeyAckOnce blocks until the resource-key-bind broadcast message is acked at any one vchannel.
|
||||
BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error
|
||||
|
||||
// BlockUntilResourceKeyAckOnce blocks until the resource-key-bind broadcast message is acked at all vchannel.
|
||||
BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error
|
||||
}
|
||||
|
||||
// Txn is the interface for writing transaction into the wal.
|
||||
|
||||
@ -16,7 +16,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/options"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
_ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/pulsar"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
@ -40,9 +39,6 @@ func TestStreamingBroadcast(t *testing.T) {
|
||||
streaming.Init()
|
||||
defer streaming.Release()
|
||||
|
||||
err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(context.Background(), message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.NoError(t, err)
|
||||
|
||||
msg, _ := message.NewCreateCollectionMessageBuilderV1().
|
||||
WithHeader(&message.CreateCollectionMessageHeader{
|
||||
CollectionId: 1,
|
||||
@ -69,53 +65,6 @@ func TestStreamingBroadcast(t *testing.T) {
|
||||
assert.Error(t, err)
|
||||
assert.True(t, status.AsStreamingError(err).IsResourceAcquired())
|
||||
assert.Nil(t, resp2)
|
||||
|
||||
// resource key should be block until ack.
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
defer cancel()
|
||||
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
defer cancel()
|
||||
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
|
||||
err = streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{
|
||||
BroadcastID: resp.BroadcastID,
|
||||
VChannel: vChannels[0],
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
|
||||
// all should be blocked
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
defer cancel()
|
||||
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
|
||||
// once should be returned
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
defer cancel()
|
||||
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{
|
||||
BroadcastID: resp.BroadcastID,
|
||||
VChannel: vChannels[1],
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
|
||||
// all should be blocked
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
defer cancel()
|
||||
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.NoError(t, err)
|
||||
|
||||
// once should be returned
|
||||
ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100)
|
||||
defer cancel()
|
||||
err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName))
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
|
||||
func TestStreamingProduce(t *testing.T) {
|
||||
|
||||
@ -46,7 +46,6 @@ func TestWAL(t *testing.T) {
|
||||
}, nil
|
||||
})
|
||||
broadcastServce.EXPECT().Ack(mock.Anything, mock.Anything).Return(nil)
|
||||
broadcastServce.EXPECT().BlockUntilEvent(mock.Anything, mock.Anything).Return(nil)
|
||||
coordClient.EXPECT().Broadcast().Return(broadcastServce)
|
||||
handler := mock_handler.NewMockHandlerClient(t)
|
||||
handler.EXPECT().Close().Return()
|
||||
@ -139,12 +138,6 @@ func TestWAL(t *testing.T) {
|
||||
err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1"))
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2"))
|
||||
assert.NoError(t, err)
|
||||
|
||||
w.Close()
|
||||
|
||||
w.GetLatestMVCCTimestampIfLocal(ctx, vChannel1)
|
||||
@ -158,12 +151,6 @@ func TestWAL(t *testing.T) {
|
||||
|
||||
err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1})
|
||||
assert.Error(t, err)
|
||||
|
||||
err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1"))
|
||||
assert.Error(t, err)
|
||||
|
||||
err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2"))
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func newInsertMessage(vChannel string) message.MutableMessage {
|
||||
|
||||
@ -130,100 +130,6 @@ func (_c *MockBroadcast_Append_Call) RunAndReturn(run func(context.Context, mess
|
||||
return _c
|
||||
}
|
||||
|
||||
// BlockUntilResourceKeyAckAll provides a mock function with given fields: ctx, rk
|
||||
func (_m *MockBroadcast) BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error {
|
||||
ret := _m.Called(ctx, rk)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for BlockUntilResourceKeyAckAll")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok {
|
||||
r0 = rf(ctx, rk)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockBroadcast_BlockUntilResourceKeyAckAll_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckAll'
|
||||
type MockBroadcast_BlockUntilResourceKeyAckAll_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// BlockUntilResourceKeyAckAll is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - rk message.ResourceKey
|
||||
func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckAll(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
|
||||
return &MockBroadcast_BlockUntilResourceKeyAckAll_Call{Call: _e.mock.On("BlockUntilResourceKeyAckAll", ctx, rk)}
|
||||
}
|
||||
|
||||
func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(message.ResourceKey))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// BlockUntilResourceKeyAckOnce provides a mock function with given fields: ctx, rk
|
||||
func (_m *MockBroadcast) BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error {
|
||||
ret := _m.Called(ctx, rk)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for BlockUntilResourceKeyAckOnce")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok {
|
||||
r0 = rf(ctx, rk)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockBroadcast_BlockUntilResourceKeyAckOnce_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckOnce'
|
||||
type MockBroadcast_BlockUntilResourceKeyAckOnce_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// BlockUntilResourceKeyAckOnce is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - rk message.ResourceKey
|
||||
func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckOnce(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
|
||||
return &MockBroadcast_BlockUntilResourceKeyAckOnce_Call{Call: _e.mock.On("BlockUntilResourceKeyAckOnce", ctx, rk)}
|
||||
}
|
||||
|
||||
func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(message.ResourceKey))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockBroadcast creates a new instance of MockBroadcast. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockBroadcast(t interface {
|
||||
|
||||
@ -5,9 +5,7 @@ package mock_client
|
||||
import (
|
||||
context "context"
|
||||
|
||||
messagespb "github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
|
||||
types "github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
@ -73,53 +71,6 @@ func (_c *MockBroadcastService_Ack_Call) RunAndReturn(run func(context.Context,
|
||||
return _c
|
||||
}
|
||||
|
||||
// BlockUntilEvent provides a mock function with given fields: ctx, ev
|
||||
func (_m *MockBroadcastService) BlockUntilEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error {
|
||||
ret := _m.Called(ctx, ev)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for BlockUntilEvent")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *messagespb.BroadcastEvent) error); ok {
|
||||
r0 = rf(ctx, ev)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockBroadcastService_BlockUntilEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilEvent'
|
||||
type MockBroadcastService_BlockUntilEvent_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// BlockUntilEvent is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - ev *messagespb.BroadcastEvent
|
||||
func (_e *MockBroadcastService_Expecter) BlockUntilEvent(ctx interface{}, ev interface{}) *MockBroadcastService_BlockUntilEvent_Call {
|
||||
return &MockBroadcastService_BlockUntilEvent_Call{Call: _e.mock.On("BlockUntilEvent", ctx, ev)}
|
||||
}
|
||||
|
||||
func (_c *MockBroadcastService_BlockUntilEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockBroadcastService_BlockUntilEvent_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroadcastService_BlockUntilEvent_Call) Return(_a0 error) *MockBroadcastService_BlockUntilEvent_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroadcastService_BlockUntilEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockBroadcastService_BlockUntilEvent_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Broadcast provides a mock function with given fields: ctx, msg
|
||||
func (_m *MockBroadcastService) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
|
||||
ret := _m.Called(ctx, msg)
|
||||
|
||||
@ -41,7 +41,6 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/http"
|
||||
"github.com/milvus-io/milvus/internal/proxy/connection"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
@ -56,7 +55,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/crypto"
|
||||
@ -6573,21 +6571,6 @@ func (node *Proxy) ImportV2(ctx context.Context, req *internalpb.ImportRequest)
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
// Import is asynchronous consumed from the wal, so we need to wait for the wal to release the resource key.
|
||||
// The job can be seen by the user after the resource key is acked once at any vchannel.
|
||||
jobID, err := strconv.ParseInt(resp.GetJobID(), 10, 64)
|
||||
if err != nil {
|
||||
return &internalpb.ImportResponse{
|
||||
Status: merr.Status(merr.WrapErrServiceInternal("invalid job ID")),
|
||||
}, nil
|
||||
}
|
||||
resourceKey := message.NewImportJobIDResourceKey(jobID)
|
||||
if err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, resourceKey); err != nil {
|
||||
log.Warn("failed to wait for resource key ack", zap.Error(err))
|
||||
return &internalpb.ImportResponse{
|
||||
Status: merr.Status(merr.WrapErrServiceInternal("failed to wait for resource key ack")),
|
||||
}, nil
|
||||
}
|
||||
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.SuccessLabel, req.GetDbName(), req.GetCollectionName()).Inc()
|
||||
metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||
return resp, nil
|
||||
|
||||
@ -1617,7 +1617,6 @@ func TestProxy_ImportV2(t *testing.T) {
|
||||
wal := mock_streaming.NewMockWALAccesser(t)
|
||||
b := mock_streaming.NewMockBroadcast(t)
|
||||
wal.EXPECT().Broadcast().Return(b).Maybe()
|
||||
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Maybe()
|
||||
streaming.SetWALForTest(wal)
|
||||
defer streaming.RecoverWALForTest()
|
||||
@ -1649,7 +1648,6 @@ func TestProxy_ImportV2(t *testing.T) {
|
||||
err = node.sched.Start()
|
||||
assert.NoError(t, err)
|
||||
chMgr := NewMockChannelsMgr(t)
|
||||
chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil)
|
||||
node.chMgr = chMgr
|
||||
|
||||
// no such collection
|
||||
@ -1686,7 +1684,6 @@ func TestProxy_ImportV2(t *testing.T) {
|
||||
// set partition name and with partition key
|
||||
chMgr = NewMockChannelsMgr(t)
|
||||
chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"ch0"}, nil)
|
||||
chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil)
|
||||
node.chMgr = chMgr
|
||||
rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{CollectionName: "aaa", PartitionName: "bbb"})
|
||||
assert.NoError(t, err)
|
||||
|
||||
@ -4660,7 +4660,6 @@ func TestProxy_Import(t *testing.T) {
|
||||
wal := mock_streaming.NewMockWALAccesser(t)
|
||||
b := mock_streaming.NewMockBroadcast(t)
|
||||
wal.EXPECT().Broadcast().Return(b).Maybe()
|
||||
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
streaming.SetWALForTest(wal)
|
||||
defer streaming.RecoverWALForTest()
|
||||
|
||||
@ -4689,7 +4688,6 @@ func TestProxy_Import(t *testing.T) {
|
||||
|
||||
chMgr := NewMockChannelsMgr(t)
|
||||
chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"foo"}, nil)
|
||||
chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"foo_v1"}, nil)
|
||||
proxy.chMgr = chMgr
|
||||
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
@ -4716,7 +4714,6 @@ func TestProxy_Import(t *testing.T) {
|
||||
b := mock_streaming.NewMockBroadcast(t)
|
||||
wal.EXPECT().Broadcast().Return(b)
|
||||
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil)
|
||||
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
streaming.SetWALForTest(wal)
|
||||
defer streaming.RecoverWALForTest()
|
||||
|
||||
|
||||
@ -199,20 +199,13 @@ func (it *importTask) PreExecute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
func (it *importTask) setChannels() error {
|
||||
collID, err := globalMetaCache.GetCollectionID(it.ctx, it.req.GetDbName(), it.req.CollectionName)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
channels, err := it.node.chMgr.getChannels(collID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
it.pchannels = channels
|
||||
// import task only send message by broadcast, which didn't affect the time tick rule at proxy.
|
||||
// so we don't need to set channels here.
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it *importTask) getChannels() []pChan {
|
||||
return it.pchannels
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it *importTask) Execute(ctx context.Context) error {
|
||||
|
||||
@ -32,6 +32,7 @@ func (b *broadcastTask) Execute(ctx context.Context) error {
|
||||
|
||||
for idx, msg := range b.msgs {
|
||||
tsMsg, err := adaptor.NewMsgPackFromMutableMessageV1(msg)
|
||||
tsMsg.SetTs(b.ts) // overwrite the ts.
|
||||
if err != nil {
|
||||
result.FillResponseAtIdx(types.AppendResponse{Error: err}, idx)
|
||||
return err
|
||||
|
||||
@ -2,35 +2,19 @@ package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
var logger = log.With(log.FieldComponent("broadcast-client"))
|
||||
|
||||
// NewGRPCBroadcastService creates a new broadcast service with grpc.
|
||||
func NewGRPCBroadcastService(walName string, service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]) *GRPCBroadcastServiceImpl {
|
||||
rw := newResumingWatcher(&grpcWatcherBuilder{
|
||||
broadcastService: service,
|
||||
}, &typeutil.BackoffTimerConfig{
|
||||
Default: 5 * time.Second,
|
||||
Backoff: typeutil.BackoffConfig{
|
||||
InitialInterval: 50 * time.Millisecond,
|
||||
Multiplier: 2.0,
|
||||
MaxInterval: 5 * time.Second,
|
||||
},
|
||||
})
|
||||
return &GRPCBroadcastServiceImpl{
|
||||
walName: walName,
|
||||
service: service,
|
||||
w: rw,
|
||||
}
|
||||
}
|
||||
|
||||
@ -39,7 +23,6 @@ func NewGRPCBroadcastService(walName string, service lazygrpc.Service[streamingp
|
||||
type GRPCBroadcastServiceImpl struct {
|
||||
walName string
|
||||
service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]
|
||||
w *resumingWatcher
|
||||
}
|
||||
|
||||
func (c *GRPCBroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
|
||||
@ -87,10 +70,5 @@ func (c *GRPCBroadcastServiceImpl) Ack(ctx context.Context, req types.BroadcastA
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *GRPCBroadcastServiceImpl) BlockUntilEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
return c.w.ObserveResourceKeyEvent(ctx, ev)
|
||||
}
|
||||
|
||||
func (c *GRPCBroadcastServiceImpl) Close() {
|
||||
c.w.Close()
|
||||
}
|
||||
|
||||
@ -3,10 +3,17 @@ package broadcast
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/mocks/proto/mock_streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
||||
@ -27,7 +34,23 @@ func TestBroadcast(t *testing.T) {
|
||||
BroadcastID: 1,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
err = bs.BlockUntilEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("r1")))
|
||||
assert.NoError(t, err)
|
||||
bs.Close()
|
||||
}
|
||||
|
||||
func newMockServer(t *testing.T, sendDelay time.Duration) lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] {
|
||||
s := mock_lazygrpc.NewMockService[streamingpb.StreamingCoordBroadcastServiceClient](t)
|
||||
c := mock_streamingpb.NewMockStreamingCoordBroadcastServiceClient(t)
|
||||
s.EXPECT().GetService(mock.Anything).Return(c, nil)
|
||||
c.EXPECT().Broadcast(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastResponse{
|
||||
Results: map[string]*streamingpb.ProduceMessageResponseResult{
|
||||
"v1": {
|
||||
Id: &messagespb.MessageID{
|
||||
Id: walimplstest.NewTestMessageID(1).Marshal(),
|
||||
},
|
||||
},
|
||||
},
|
||||
BroadcastId: 1,
|
||||
}, nil).Maybe()
|
||||
c.EXPECT().Ack(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastAckResponse{}, nil).Maybe()
|
||||
return s
|
||||
}
|
||||
|
||||
@ -1,192 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type grpcWatcherBuilder struct {
|
||||
broadcastService lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]
|
||||
}
|
||||
|
||||
func (b *grpcWatcherBuilder) Build(ctx context.Context) (Watcher, error) {
|
||||
service, err := b.broadcastService.GetService(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to get broadcast service")
|
||||
}
|
||||
bt := syncutil.NewAsyncTaskNotifier[struct{}]()
|
||||
// TODO: Here we make a broken stream by passing a context.
|
||||
// Implement a graceful closing should be better.
|
||||
streamCtx, cancel := context.WithCancel(context.Background())
|
||||
svr, err := service.Watch(streamCtx)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, errors.Wrap(err, "failed to create broadcast watcher server client")
|
||||
}
|
||||
w := &grpcWatcherClient{
|
||||
lifetime: typeutil.NewLifetime(),
|
||||
backgroundTask: bt,
|
||||
streamServerCancel: cancel,
|
||||
streamClient: svr,
|
||||
input: make(chan *message.BroadcastEvent),
|
||||
output: make(chan *message.BroadcastEvent),
|
||||
sendExitCh: make(chan struct{}),
|
||||
recvExitCh: make(chan struct{}),
|
||||
}
|
||||
w.SetLogger(logger)
|
||||
go w.executeBackgroundTask()
|
||||
return w, nil
|
||||
}
|
||||
|
||||
type grpcWatcherClient struct {
|
||||
log.Binder
|
||||
lifetime *typeutil.Lifetime
|
||||
backgroundTask *syncutil.AsyncTaskNotifier[struct{}]
|
||||
streamServerCancel context.CancelFunc
|
||||
streamClient streamingpb.StreamingCoordBroadcastService_WatchClient
|
||||
input chan *message.BroadcastEvent
|
||||
output chan *message.BroadcastEvent
|
||||
recvExitCh chan struct{}
|
||||
sendExitCh chan struct{}
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
if !c.lifetime.Add(typeutil.LifetimeStateWorking) {
|
||||
return errWatcherClosed
|
||||
}
|
||||
defer c.lifetime.Done()
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-c.backgroundTask.Context().Done():
|
||||
return c.backgroundTask.Context().Err()
|
||||
case c.input <- ev:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) EventChan() <-chan *message.BroadcastEvent {
|
||||
return c.output
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) gracefulClose() error {
|
||||
c.lifetime.SetState(typeutil.LifetimeStateStopped)
|
||||
// cancel the background task and wait for all request to finish
|
||||
c.backgroundTask.Cancel()
|
||||
c.lifetime.Wait()
|
||||
|
||||
select {
|
||||
case <-c.backgroundTask.FinishChan():
|
||||
return nil
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
return context.DeadlineExceeded
|
||||
}
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) Close() {
|
||||
// Try to make a graceful close.
|
||||
if err := c.gracefulClose(); err != nil {
|
||||
c.Logger().Warn("failed to close the broadcast watcher gracefully, a froce closing will be applied", zap.Error(err))
|
||||
}
|
||||
c.streamServerCancel()
|
||||
c.backgroundTask.BlockUntilFinish()
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) executeBackgroundTask() {
|
||||
defer func() {
|
||||
close(c.output)
|
||||
c.backgroundTask.Finish(struct{}{})
|
||||
}()
|
||||
|
||||
go c.recvLoop()
|
||||
go c.sendLoop()
|
||||
<-c.recvExitCh
|
||||
<-c.sendExitCh
|
||||
}
|
||||
|
||||
// sendLoop send the incoming event to the remote server.
|
||||
// If the input channel is closed, it will send a close message to the remote server and return.
|
||||
func (c *grpcWatcherClient) sendLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
c.Logger().Warn("send arm of stream closed by unexpected error", zap.Error(err))
|
||||
} else {
|
||||
c.Logger().Info("send arm of stream closed")
|
||||
}
|
||||
if err := c.streamClient.CloseSend(); err != nil {
|
||||
c.Logger().Warn("failed to close send", zap.Error(err))
|
||||
}
|
||||
close(c.sendExitCh)
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-c.backgroundTask.Context().Done():
|
||||
// send close message stop the loop.
|
||||
// then the server will close the recv arm and return io.EOF.
|
||||
// recv arm can be closed after that.
|
||||
return c.streamClient.Send(&streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_Close{
|
||||
Close: &streamingpb.CloseBroadcastWatchRequest{},
|
||||
},
|
||||
})
|
||||
case ev := <-c.input:
|
||||
if err := c.streamClient.Send(&streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_CreateEventWatch{
|
||||
CreateEventWatch: &streamingpb.BroadcastCreateEventWatchRequest{
|
||||
Event: ev,
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// recvLoop receive the event from the remote server.
|
||||
func (c *grpcWatcherClient) recvLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
c.Logger().Warn("recv arm of stream closed by unexpected error", zap.Error(err))
|
||||
} else {
|
||||
c.Logger().Info("recv arm of stream closed")
|
||||
}
|
||||
close(c.recvExitCh)
|
||||
}()
|
||||
|
||||
for {
|
||||
resp, err := c.streamClient.Recv()
|
||||
if errors.Is(err, io.EOF) {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch resp := resp.Response.(type) {
|
||||
case *streamingpb.BroadcastWatchResponse_EventDone:
|
||||
select {
|
||||
case c.output <- resp.EventDone.Event:
|
||||
case <-c.backgroundTask.Context().Done():
|
||||
c.Logger().Info("recv arm close when send event to output channel, skip wait for io.EOF")
|
||||
return nil
|
||||
}
|
||||
case *streamingpb.BroadcastWatchResponse_Close:
|
||||
// nothing to do now, just wait io.EOF.
|
||||
default:
|
||||
c.Logger().Warn("unknown response type", zap.Any("response", resp))
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1,125 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/mocks/proto/mock_streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
||||
)
|
||||
|
||||
func TestWatcher(t *testing.T) {
|
||||
s := newMockServer(t, 0)
|
||||
|
||||
b := grpcWatcherBuilder{broadcastService: s}
|
||||
w, err := b.Build(context.Background())
|
||||
assert.NoError(t, err)
|
||||
|
||||
done := make(chan struct{})
|
||||
cnt := 0
|
||||
go func() {
|
||||
defer close(done)
|
||||
for range w.EventChan() {
|
||||
cnt++
|
||||
}
|
||||
}()
|
||||
for i := 0; i < 10; i++ {
|
||||
err = w.ObserveResourceKeyEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
w.Close()
|
||||
<-done
|
||||
assert.Equal(t, 10, cnt)
|
||||
err = w.ObserveResourceKeyEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assert.Error(t, err)
|
||||
|
||||
// Test ungraceful close
|
||||
s = newMockServer(t, 10*time.Second)
|
||||
b2 := grpcWatcherBuilder{broadcastService: s}
|
||||
w2, err := b2.Build(context.Background())
|
||||
assert.NoError(t, err)
|
||||
w2.Close()
|
||||
}
|
||||
|
||||
func newMockServer(t *testing.T, sendDelay time.Duration) lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] {
|
||||
s := mock_lazygrpc.NewMockService[streamingpb.StreamingCoordBroadcastServiceClient](t)
|
||||
c := mock_streamingpb.NewMockStreamingCoordBroadcastServiceClient(t)
|
||||
s.EXPECT().GetService(mock.Anything).Return(c, nil)
|
||||
var ctx context.Context
|
||||
cc := mock_streamingpb.NewMockStreamingCoordBroadcastService_WatchClient(t)
|
||||
c.EXPECT().Watch(mock.Anything).RunAndReturn(func(ctx2 context.Context, co ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error) {
|
||||
ctx = ctx2
|
||||
return cc, nil
|
||||
})
|
||||
c.EXPECT().Broadcast(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastResponse{
|
||||
Results: map[string]*streamingpb.ProduceMessageResponseResult{
|
||||
"v1": {
|
||||
Id: &messagespb.MessageID{
|
||||
Id: walimplstest.NewTestMessageID(1).Marshal(),
|
||||
},
|
||||
},
|
||||
},
|
||||
BroadcastId: 1,
|
||||
}, nil).Maybe()
|
||||
c.EXPECT().Ack(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastAckResponse{}, nil).Maybe()
|
||||
|
||||
output := make(chan *streamingpb.BroadcastWatchRequest, 10)
|
||||
cc.EXPECT().Recv().RunAndReturn(func() (*streamingpb.BroadcastWatchResponse, error) {
|
||||
var result *streamingpb.BroadcastWatchRequest
|
||||
var ok bool
|
||||
select {
|
||||
case result, ok = <-output:
|
||||
if !ok {
|
||||
return nil, io.EOF
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
switch cmd := result.Command.(type) {
|
||||
case *streamingpb.BroadcastWatchRequest_Close:
|
||||
return &streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_Close{Close: &streamingpb.CloseBroadcastWatchResponse{}},
|
||||
}, nil
|
||||
case *streamingpb.BroadcastWatchRequest_CreateEventWatch:
|
||||
return &streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_EventDone{
|
||||
EventDone: &streamingpb.BroadcastEventWatchResponse{
|
||||
Event: cmd.CreateEventWatch.Event,
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
default:
|
||||
panic("unknown command")
|
||||
}
|
||||
})
|
||||
cc.EXPECT().Send(mock.Anything).RunAndReturn(func(bwr *streamingpb.BroadcastWatchRequest) error {
|
||||
select {
|
||||
case <-time.After(sendDelay):
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
select {
|
||||
case output <- bwr:
|
||||
return nil
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
})
|
||||
cc.EXPECT().CloseSend().RunAndReturn(func() error {
|
||||
close(output)
|
||||
return nil
|
||||
})
|
||||
return s
|
||||
}
|
||||
@ -1,22 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
type WatcherBuilder interface {
|
||||
Build(ctx context.Context) (Watcher, error)
|
||||
}
|
||||
|
||||
type Watcher interface {
|
||||
// ObserveResourceKeyEvent observes the resource key event.
|
||||
ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error
|
||||
|
||||
// EventChan returns the event channel.
|
||||
EventChan() <-chan *message.BroadcastEvent
|
||||
|
||||
// Close closes the watcher.
|
||||
Close()
|
||||
}
|
||||
@ -1,178 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
var errWatcherClosed = errors.New("watcher is closed")
|
||||
|
||||
// newResumingWatcher create a new resuming watcher.
|
||||
func newResumingWatcher(b WatcherBuilder, backoffConfig *typeutil.BackoffTimerConfig) *resumingWatcher {
|
||||
rw := &resumingWatcher{
|
||||
backgroundTask: syncutil.NewAsyncTaskNotifier[struct{}](),
|
||||
input: make(chan *pendingEvent),
|
||||
evs: &pendingEvents{evs: make(map[string]*pendingEvent)},
|
||||
watcherBuilder: b, // TODO: enable local watcher here.
|
||||
}
|
||||
rw.SetLogger(logger)
|
||||
go rw.execute(backoffConfig)
|
||||
return rw
|
||||
}
|
||||
|
||||
// resumingWatcher is a watcher that can resume the watcher when it is unavailable.
|
||||
type resumingWatcher struct {
|
||||
log.Binder
|
||||
backgroundTask *syncutil.AsyncTaskNotifier[struct{}]
|
||||
input chan *pendingEvent
|
||||
evs *pendingEvents
|
||||
watcherBuilder WatcherBuilder
|
||||
}
|
||||
|
||||
// ObserveResourceKeyEvent observes the resource key event.
|
||||
func (r *resumingWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
notifier := make(chan struct{})
|
||||
select {
|
||||
case <-r.backgroundTask.Context().Done():
|
||||
return errWatcherClosed
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case r.input <- &pendingEvent{
|
||||
ev: ev,
|
||||
notifier: []chan<- struct{}{notifier},
|
||||
}:
|
||||
}
|
||||
select {
|
||||
case <-r.backgroundTask.Context().Done():
|
||||
return errWatcherClosed
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-notifier:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (r *resumingWatcher) Close() {
|
||||
r.backgroundTask.Cancel()
|
||||
r.backgroundTask.BlockUntilFinish()
|
||||
}
|
||||
|
||||
func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
|
||||
backoff := typeutil.NewBackoffTimer(backoffConfig)
|
||||
var nextTimer <-chan time.Time
|
||||
var initialized bool
|
||||
var watcher Watcher
|
||||
defer func() {
|
||||
if watcher != nil {
|
||||
watcher.Close()
|
||||
}
|
||||
r.backgroundTask.Finish(struct{}{})
|
||||
}()
|
||||
|
||||
for {
|
||||
var eventChan <-chan *message.BroadcastEvent
|
||||
if watcher != nil {
|
||||
eventChan = watcher.EventChan()
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.backgroundTask.Context().Done():
|
||||
return
|
||||
case ev := <-r.input:
|
||||
if !r.evs.AddPendingEvent(ev) && watcher != nil {
|
||||
if err := watcher.ObserveResourceKeyEvent(r.backgroundTask.Context(), ev.ev); err != nil {
|
||||
watcher.Close()
|
||||
watcher = nil
|
||||
}
|
||||
}
|
||||
if !initialized {
|
||||
// try to initialize watcher in next loop.
|
||||
// avoid to make a grpc stream channel if the watch operation is not used.
|
||||
nextTimer = time.After(0)
|
||||
initialized = true
|
||||
}
|
||||
case ev, ok := <-eventChan:
|
||||
if !ok {
|
||||
watcher.Close()
|
||||
watcher = nil
|
||||
break
|
||||
}
|
||||
r.evs.Notify(ev)
|
||||
case <-nextTimer:
|
||||
var err error
|
||||
nextTimer = nil
|
||||
if watcher, err = r.createNewWatcher(); err != nil {
|
||||
r.Logger().Warn("create new watcher failed", zap.Error(err))
|
||||
break
|
||||
}
|
||||
r.Logger().Info("create new watcher successful")
|
||||
backoff.DisableBackoff()
|
||||
}
|
||||
if watcher == nil && nextTimer == nil {
|
||||
backoff.EnableBackoff()
|
||||
var interval time.Duration
|
||||
nextTimer, interval = backoff.NextTimer()
|
||||
r.Logger().Warn("watcher is unavailable, resuming it after interval", zap.Duration("interval", interval))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (r *resumingWatcher) createNewWatcher() (Watcher, error) {
|
||||
watcher, err := r.watcherBuilder.Build(r.backgroundTask.Context())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := r.evs.SendAll(r.backgroundTask.Context(), watcher); err != nil {
|
||||
watcher.Close()
|
||||
return nil, errors.Wrapf(err, "send all pending events to watcher failed")
|
||||
}
|
||||
return watcher, nil
|
||||
}
|
||||
|
||||
type pendingEvents struct {
|
||||
evs map[string]*pendingEvent
|
||||
}
|
||||
|
||||
// AddPendingEvent adds a pending event.
|
||||
// Return true if the event is already in the pending events.
|
||||
func (evs *pendingEvents) AddPendingEvent(ev *pendingEvent) bool {
|
||||
id := message.UniqueKeyOfBroadcastEvent(ev.ev)
|
||||
if existEv, ok := evs.evs[id]; ok {
|
||||
existEv.notifier = append(existEv.notifier, ev.notifier...)
|
||||
return true
|
||||
}
|
||||
evs.evs[id] = ev
|
||||
return false
|
||||
}
|
||||
|
||||
func (evs *pendingEvents) Notify(ev *message.BroadcastEvent) {
|
||||
id := message.UniqueKeyOfBroadcastEvent(ev)
|
||||
if existEv, ok := evs.evs[id]; ok {
|
||||
for _, notifier := range existEv.notifier {
|
||||
close(notifier)
|
||||
}
|
||||
delete(evs.evs, id)
|
||||
}
|
||||
}
|
||||
|
||||
func (evs *pendingEvents) SendAll(ctx context.Context, w Watcher) error {
|
||||
for _, ev := range evs.evs {
|
||||
if err := w.ObserveResourceKeyEvent(ctx, ev.ev); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type pendingEvent struct {
|
||||
ev *message.BroadcastEvent
|
||||
notifier []chan<- struct{}
|
||||
}
|
||||
@ -1,101 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"go.uber.org/atomic"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/client/mock_broadcast"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type mockBuilder struct {
|
||||
built func(ctx context.Context) (Watcher, error)
|
||||
}
|
||||
|
||||
func (b *mockBuilder) Build(ctx context.Context) (Watcher, error) {
|
||||
return b.built(ctx)
|
||||
}
|
||||
|
||||
func TestWatcherResuming(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
b := newMockWatcherBuilder(t)
|
||||
rw := newResumingWatcher(b, &typeutil.BackoffTimerConfig{
|
||||
Default: 500 * time.Millisecond,
|
||||
Backoff: typeutil.BackoffConfig{
|
||||
InitialInterval: 10 * time.Millisecond,
|
||||
Multiplier: 2.0,
|
||||
MaxInterval: 500 * time.Millisecond,
|
||||
},
|
||||
})
|
||||
wg := &sync.WaitGroup{}
|
||||
for i := 0; i < 100; i++ {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
id := rand.Int31n(10)
|
||||
rk := message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey(fmt.Sprintf("c%d", id)))
|
||||
err := rw.ObserveResourceKeyEvent(ctx, rk)
|
||||
assert.NoError(t, err)
|
||||
}()
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
rw.Close()
|
||||
err := rw.ObserveResourceKeyEvent(ctx, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assert.ErrorIs(t, err, errWatcherClosed)
|
||||
}
|
||||
|
||||
func newMockWatcherBuilder(t *testing.T) WatcherBuilder {
|
||||
return &mockBuilder{built: func(ctx context.Context) (Watcher, error) {
|
||||
w := mock_broadcast.NewMockWatcher(t)
|
||||
n := rand.Int31n(10)
|
||||
if n < 3 {
|
||||
return nil, errors.New("err")
|
||||
}
|
||||
|
||||
// ill watcher
|
||||
k := atomic.NewInt32(n)
|
||||
o := rand.Int31n(20) + n
|
||||
mu := sync.Mutex{}
|
||||
closed := false
|
||||
output := make(chan *message.BroadcastEvent, 500)
|
||||
w.EXPECT().ObserveResourceKeyEvent(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, be *messagespb.BroadcastEvent) error {
|
||||
k2 := k.Inc()
|
||||
if k2 >= o {
|
||||
return errors.New("err")
|
||||
}
|
||||
mu.Lock()
|
||||
if closed {
|
||||
return errors.New("closed")
|
||||
}
|
||||
go func() {
|
||||
defer mu.Unlock()
|
||||
time.Sleep(time.Duration(rand.Int31n(5)) * time.Millisecond)
|
||||
output <- be
|
||||
}()
|
||||
return nil
|
||||
}).Maybe()
|
||||
w.EXPECT().EventChan().RunAndReturn(func() <-chan *messagespb.BroadcastEvent {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
if !closed && rand.Int31n(100) < 50 {
|
||||
close(output)
|
||||
closed = true
|
||||
}
|
||||
return output
|
||||
}).Maybe()
|
||||
w.EXPECT().Close().Return()
|
||||
return w, nil
|
||||
}}
|
||||
}
|
||||
@ -44,9 +44,6 @@ type BroadcastService interface {
|
||||
// Ack sends a broadcast ack to the streaming service.
|
||||
Ack(ctx context.Context, req types.BroadcastAckRequest) error
|
||||
|
||||
// BlockUntilEvent blocks until the event happens.
|
||||
BlockUntilEvent(ctx context.Context, ev *message.BroadcastEvent) error
|
||||
|
||||
// Close closes the broadcast service.
|
||||
Close()
|
||||
}
|
||||
|
||||
@ -9,7 +9,6 @@ import (
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
@ -72,13 +71,14 @@ func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.Broadca
|
||||
if msg, err = bm.assignID(ctx, msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
task, err := bm.addBroadcastTask(msg)
|
||||
task, err := bm.addBroadcastTask(ctx, msg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return newPendingBroadcastTask(task), nil
|
||||
}
|
||||
|
||||
// assignID assigns the broadcast id to the message.
|
||||
func (bm *broadcastTaskManager) assignID(ctx context.Context, msg message.BroadcastMutableMessage) (message.BroadcastMutableMessage, error) {
|
||||
// TODO: current implementation the header cannot be seen at flusher itself.
|
||||
// only import message use it, so temporarily set the broadcast id here.
|
||||
@ -116,72 +116,56 @@ func (bm *broadcastTaskManager) Ack(ctx context.Context, broadcastID uint64, vch
|
||||
|
||||
if task.State() == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE {
|
||||
bm.removeBroadcastTask(broadcastID)
|
||||
} else {
|
||||
bm.increaseVersion()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// WatchAtVersion watches the version of the broadcast task manager.
|
||||
// When the version is greater than the input version, the watcher will be notified.
|
||||
func (bm *broadcastTaskManager) WatchAtVersion(version int) <-chan struct{} {
|
||||
bm.cond.L.Lock()
|
||||
if bm.version > version {
|
||||
bm.cond.L.Unlock()
|
||||
ch := make(chan struct{})
|
||||
close(ch)
|
||||
return ch
|
||||
}
|
||||
return bm.cond.WaitChan()
|
||||
}
|
||||
|
||||
// CurrentVersion returns the current version of the broadcast task manager.
|
||||
func (bm *broadcastTaskManager) CurrentVersion() int {
|
||||
bm.cond.L.Lock()
|
||||
defer bm.cond.L.Unlock()
|
||||
return bm.version
|
||||
}
|
||||
|
||||
// GetBroadcastTaskByResourceKey returns the broadcast task by the resource key.
|
||||
func (bm *broadcastTaskManager) GetBroadcastTaskByResourceKey(resourceKey message.ResourceKey) (*broadcastTask, bool) {
|
||||
bm.cond.L.Lock()
|
||||
// ReleaseResourceKeys releases the resource keys by the broadcastID.
|
||||
func (bm *broadcastTaskManager) ReleaseResourceKeys(broadcastID uint64) {
|
||||
bm.cond.LockAndBroadcast()
|
||||
defer bm.cond.L.Unlock()
|
||||
|
||||
broadcastID, ok := bm.resourceKeys[resourceKey]
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
task, ok := bm.tasks[broadcastID]
|
||||
return task, ok
|
||||
bm.removeResourceKeys(broadcastID)
|
||||
}
|
||||
|
||||
// addBroadcastTask adds the broadcast task into the manager.
|
||||
func (bm *broadcastTaskManager) addBroadcastTask(msg message.BroadcastMutableMessage) (*broadcastTask, error) {
|
||||
func (bm *broadcastTaskManager) addBroadcastTask(ctx context.Context, msg message.BroadcastMutableMessage) (*broadcastTask, error) {
|
||||
newIncomingTask := newBroadcastTaskFromBroadcastMessage(msg, bm.metrics)
|
||||
header := newIncomingTask.Header()
|
||||
newIncomingTask.SetLogger(bm.Logger().With(zap.Uint64("broadcastID", header.BroadcastID)))
|
||||
|
||||
bm.cond.L.Lock()
|
||||
defer bm.cond.L.Unlock()
|
||||
// Check if the resource key is held by other task.
|
||||
for key := range header.ResourceKeys {
|
||||
if _, ok := bm.resourceKeys[key]; ok {
|
||||
return nil, status.NewResourceAcquired(fmt.Sprintf("domain: %s, key: %s", key.Domain.String(), key.Key))
|
||||
for bm.checkIfResourceKeyExist(header) {
|
||||
if err := bm.cond.Wait(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// setup the resource keys to make resource exclusive held.
|
||||
for key := range header.ResourceKeys {
|
||||
bm.resourceKeys[key] = header.BroadcastID
|
||||
bm.metrics.IncomingResourceKey(key.Domain)
|
||||
}
|
||||
bm.tasks[header.BroadcastID] = newIncomingTask
|
||||
bm.cond.L.Unlock()
|
||||
// TODO: perform a task checker here to make sure the task is vaild to be broadcasted in future.
|
||||
return newIncomingTask, nil
|
||||
}
|
||||
|
||||
func (bm *broadcastTaskManager) checkIfResourceKeyExist(header *message.BroadcastHeader) bool {
|
||||
for key := range header.ResourceKeys {
|
||||
if _, ok := bm.resourceKeys[key]; ok {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// getBroadcastTaskByID return the task by the broadcastID.
|
||||
func (bm *broadcastTaskManager) getBroadcastTaskByID(broadcastID uint64) (*broadcastTask, bool) {
|
||||
bm.cond.L.Lock()
|
||||
defer bm.cond.L.Unlock()
|
||||
|
||||
t, ok := bm.tasks[broadcastID]
|
||||
return t, ok
|
||||
}
|
||||
@ -191,21 +175,19 @@ func (bm *broadcastTaskManager) removeBroadcastTask(broadcastID uint64) {
|
||||
bm.cond.LockAndBroadcast()
|
||||
defer bm.cond.L.Unlock()
|
||||
|
||||
bm.removeResourceKeys(broadcastID)
|
||||
delete(bm.tasks, broadcastID)
|
||||
}
|
||||
|
||||
// removeResourceKeys removes the resource keys by the broadcastID.
|
||||
func (bm *broadcastTaskManager) removeResourceKeys(broadcastID uint64) {
|
||||
task, ok := bm.tasks[broadcastID]
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
bm.version++
|
||||
// remove the related resource keys
|
||||
for key := range task.header.ResourceKeys {
|
||||
delete(bm.resourceKeys, key)
|
||||
bm.metrics.GoneResourceKey(key.Domain)
|
||||
}
|
||||
delete(bm.tasks, broadcastID)
|
||||
}
|
||||
|
||||
func (bm *broadcastTaskManager) increaseVersion() {
|
||||
bm.cond.LockAndBroadcast()
|
||||
bm.version++
|
||||
bm.cond.L.Unlock()
|
||||
}
|
||||
|
||||
@ -20,20 +20,25 @@ func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask, metrics *broadc
|
||||
m := metrics.NewBroadcastTask(proto.GetState())
|
||||
msg := message.NewBroadcastMutableMessageBeforeAppend(proto.Message.Payload, proto.Message.Properties)
|
||||
bh := msg.BroadcastHeader()
|
||||
return &broadcastTask{
|
||||
bt := &broadcastTask{
|
||||
mu: sync.Mutex{},
|
||||
header: bh,
|
||||
task: proto,
|
||||
recoverPersisted: true, // the task is recovered from the recovery info, so it's persisted.
|
||||
metrics: m,
|
||||
allAcked: make(chan struct{}),
|
||||
}
|
||||
if isAllDone(proto) {
|
||||
close(bt.allAcked)
|
||||
}
|
||||
return bt
|
||||
}
|
||||
|
||||
// newBroadcastTaskFromBroadcastMessage creates a new broadcast task from the broadcast message.
|
||||
func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage, metrics *broadcasterMetrics) *broadcastTask {
|
||||
m := metrics.NewBroadcastTask(streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING)
|
||||
header := msg.BroadcastHeader()
|
||||
return &broadcastTask{
|
||||
bt := &broadcastTask{
|
||||
Binder: log.Binder{},
|
||||
mu: sync.Mutex{},
|
||||
header: header,
|
||||
@ -44,7 +49,12 @@ func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage, m
|
||||
},
|
||||
recoverPersisted: false,
|
||||
metrics: m,
|
||||
allAcked: make(chan struct{}),
|
||||
}
|
||||
if isAllDone(bt.task) {
|
||||
close(bt.allAcked)
|
||||
}
|
||||
return bt
|
||||
}
|
||||
|
||||
// broadcastTask is the state of the broadcast task.
|
||||
@ -55,6 +65,7 @@ type broadcastTask struct {
|
||||
task *streamingpb.BroadcastTask
|
||||
recoverPersisted bool // a flag to indicate that the task has been persisted into the recovery info and can be recovered.
|
||||
metrics *taskMetricsGuard
|
||||
allAcked chan struct{}
|
||||
}
|
||||
|
||||
// Header returns the header of the broadcast task.
|
||||
@ -121,13 +132,23 @@ func (b *broadcastTask) Ack(ctx context.Context, vchannel string) error {
|
||||
return err
|
||||
}
|
||||
b.task = task
|
||||
b.metrics.ObserveAckAnyOne()
|
||||
if isAllDone(task) {
|
||||
b.metrics.ObserveAckAll()
|
||||
close(b.allAcked)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// BlockUntilAllAck blocks until all the vchannels are acked.
|
||||
func (b *broadcastTask) BlockUntilAllAck(ctx context.Context) error {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-b.allAcked:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// copyAndSetVChannelAcked copies the task and set the vchannel as acked.
|
||||
// if the vchannel is already acked, it returns nil and false.
|
||||
func (b *broadcastTask) copyAndSetVChannelAcked(vchannel string) (*streamingpb.BroadcastTask, bool) {
|
||||
@ -185,13 +206,6 @@ func (b *broadcastTask) copyAndMarkBroadcastDone() *streamingpb.BroadcastTask {
|
||||
return task
|
||||
}
|
||||
|
||||
// IsAllAcked returns true if all the vchannels are acked.
|
||||
func (b *broadcastTask) IsAllAcked() bool {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
return isAllDone(b.task)
|
||||
}
|
||||
|
||||
// isAllDone check if all the vchannels are acked.
|
||||
func isAllDone(task *streamingpb.BroadcastTask) bool {
|
||||
for _, acked := range task.AckedVchannelBitmap {
|
||||
@ -211,18 +225,6 @@ func ackedCount(task *streamingpb.BroadcastTask) int {
|
||||
return count
|
||||
}
|
||||
|
||||
// IsAcked returns true if any vchannel is acked.
|
||||
func (b *broadcastTask) IsAcked() bool {
|
||||
b.mu.Lock()
|
||||
defer b.mu.Unlock()
|
||||
for _, acked := range b.task.AckedVchannelBitmap {
|
||||
if acked != 0 {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// saveTask saves the broadcast task recovery info.
|
||||
func (b *broadcastTask) saveTask(ctx context.Context, task *streamingpb.BroadcastTask, logger *log.MLogger) error {
|
||||
logger = logger.With(zap.String("state", task.State.String()), zap.Int("ackedVChannelCount", ackedCount(task)))
|
||||
|
||||
@ -15,23 +15,8 @@ type Broadcaster interface {
|
||||
// Ack acknowledges the message at the specified vchannel.
|
||||
Ack(ctx context.Context, req types.BroadcastAckRequest) error
|
||||
|
||||
// Watch watches the broadcast event.
|
||||
NewWatcher() (Watcher, error)
|
||||
|
||||
// Close closes the broadcaster.
|
||||
Close()
|
||||
}
|
||||
|
||||
// Watcher is the interface for watching the broadcast event.
|
||||
type Watcher interface {
|
||||
// ObserveResourceKeyEvent observes the resource key event.
|
||||
ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error
|
||||
|
||||
// EventChan returns the event channel.
|
||||
EventChan() <-chan *message.BroadcastEvent
|
||||
|
||||
// Close closes the watcher.
|
||||
Close()
|
||||
}
|
||||
|
||||
type AppendOperator = registry.AppendOperator
|
||||
|
||||
@ -84,7 +84,16 @@ func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMu
|
||||
|
||||
// Wait both request context and the background task context.
|
||||
ctx, _ = contextutil.MergeContext(ctx, b.backgroundTaskNotifier.Context())
|
||||
return t.BlockUntilTaskDone(ctx)
|
||||
r, err := t.BlockUntilTaskDone(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// wait for all the vchannels acked.
|
||||
if err := t.BlockUntilAllAck(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// Ack acknowledges the message at the specified vchannel.
|
||||
@ -97,15 +106,6 @@ func (b *broadcasterImpl) Ack(ctx context.Context, req types.BroadcastAckRequest
|
||||
return b.manager.Ack(ctx, req.BroadcastID, req.VChannel)
|
||||
}
|
||||
|
||||
func (b *broadcasterImpl) NewWatcher() (Watcher, error) {
|
||||
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
|
||||
return nil, status.NewOnShutdownError("broadcaster is closing")
|
||||
}
|
||||
defer b.lifetime.Done()
|
||||
|
||||
return newWatcher(b), nil
|
||||
}
|
||||
|
||||
func (b *broadcasterImpl) Close() {
|
||||
b.lifetime.SetState(typeutil.LifetimeStateStopped)
|
||||
b.lifetime.Wait()
|
||||
@ -215,6 +215,8 @@ func (b *broadcasterImpl) worker(no int, appendOperator AppendOperator) {
|
||||
case b.backoffChan <- task:
|
||||
}
|
||||
}
|
||||
// All message of broadcast task is sent, release the resource keys to let other task with same resource keys to apply operation.
|
||||
b.manager.ReleaseResourceKeys(task.Header().BroadcastID)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -3,7 +3,6 @@ package broadcaster
|
||||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@ -11,7 +10,6 @@ import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"go.uber.org/atomic"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/mocks/mock_metastore"
|
||||
@ -19,7 +17,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
internaltypes "github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/idalloc"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
@ -37,8 +34,8 @@ func TestBroadcaster(t *testing.T) {
|
||||
meta.EXPECT().ListBroadcastTask(mock.Anything).
|
||||
RunAndReturn(func(ctx context.Context) ([]*streamingpb.BroadcastTask, error) {
|
||||
return []*streamingpb.BroadcastTask{
|
||||
createNewBroadcastTask(1, []string{"v1"}, message.NewCollectionNameResourceKey("c1")),
|
||||
createNewBroadcastTask(2, []string{"v1", "v2"}, message.NewCollectionNameResourceKey("c2")),
|
||||
createNewBroadcastTask(8, []string{"v1"}, message.NewCollectionNameResourceKey("c1")),
|
||||
createNewBroadcastTask(9, []string{"v1", "v2"}, message.NewCollectionNameResourceKey("c2")),
|
||||
createNewBroadcastTask(3, []string{"v1", "v2", "v3"}),
|
||||
createNewWaitAckBroadcastTaskFromMessage(
|
||||
createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(4),
|
||||
@ -76,32 +73,28 @@ func TestBroadcaster(t *testing.T) {
|
||||
f.Set(rc)
|
||||
resource.InitForTest(resource.OptStreamingCatalog(meta), resource.OptRootCoordClient(f))
|
||||
|
||||
operator, appended := createOpeartor(t)
|
||||
fbc := syncutil.NewFuture[Broadcaster]()
|
||||
operator, appended := createOpeartor(t, fbc)
|
||||
bc, err := RecoverBroadcaster(context.Background(), operator)
|
||||
fbc.Set(bc)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, bc)
|
||||
assert.Eventually(t, func() bool {
|
||||
return appended.Load() == 9 && len(done.Collect()) == 1 // only one task is done,
|
||||
return appended.Load() == 9 && len(done.Collect()) == 6 // only one task is done,
|
||||
}, 30*time.Second, 10*time.Millisecond)
|
||||
|
||||
// Test ack here
|
||||
wg := &sync.WaitGroup{}
|
||||
asyncAck(wg, bc, 1, "v1")
|
||||
asyncAck(wg, bc, 2, "v2")
|
||||
asyncAck(wg, bc, 3, "v3")
|
||||
asyncAck(wg, bc, 3, "v2")
|
||||
// repeatoperation should be ok.
|
||||
asyncAck(wg, bc, 1, "v1")
|
||||
asyncAck(wg, bc, 2, "v2")
|
||||
asyncAck(wg, bc, 3, "v3")
|
||||
asyncAck(wg, bc, 3, "v2")
|
||||
wg.Wait()
|
||||
|
||||
// only task 7 is not done.
|
||||
ack(bc, 7, "v1")
|
||||
assert.Equal(t, len(done.Collect()), 6)
|
||||
ack(bc, 7, "v2")
|
||||
assert.Equal(t, len(done.Collect()), 6)
|
||||
ack(bc, 7, "v3")
|
||||
assert.Eventually(t, func() bool {
|
||||
return len(done.Collect()) == 2
|
||||
return appended.Load() == 9 && len(done.Collect()) == 7
|
||||
}, 30*time.Second, 10*time.Millisecond)
|
||||
|
||||
// Test broadcast here.
|
||||
broadcastWithSameRK := func() {
|
||||
var result *types.BroadcastAppendResult
|
||||
for {
|
||||
var err error
|
||||
@ -110,102 +103,25 @@ func TestBroadcaster(t *testing.T) {
|
||||
break
|
||||
}
|
||||
}
|
||||
assert.Equal(t, int(appended.Load()), 12)
|
||||
assert.Equal(t, len(result.AppendResults), 3)
|
||||
}
|
||||
go broadcastWithSameRK()
|
||||
go broadcastWithSameRK()
|
||||
|
||||
assert.Eventually(t, func() bool {
|
||||
return len(done.Collect()) == 2
|
||||
return appended.Load() == 15 && len(done.Collect()) == 9
|
||||
}, 30*time.Second, 10*time.Millisecond)
|
||||
|
||||
// Test broadcast with a already exist resource key.
|
||||
for {
|
||||
var err error
|
||||
_, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}, message.NewCollectionNameResourceKey("c7")))
|
||||
if err != nil {
|
||||
assert.True(t, status.AsStreamingError(err).IsResourceAcquired())
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Test watch here.
|
||||
w, err := bc.NewWatcher()
|
||||
assert.NoError(t, err)
|
||||
// Test a resource key that not exist.
|
||||
assertResourceEventOK(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c5")))
|
||||
assertResourceEventOK(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c5")))
|
||||
// Test a resource key that already ack all.
|
||||
assertResourceEventOK(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assertResourceEventOK(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
// Test a resource key that partially ack.
|
||||
assertResourceEventOK(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c2")))
|
||||
assertResourceEventNotReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c2")))
|
||||
// Test a resource key that not ack.
|
||||
readyCh := assertResourceEventUntilReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c2")))
|
||||
ack(bc, 2, "v1")
|
||||
<-readyCh
|
||||
// Test a resource key that not ack.
|
||||
assertResourceEventNotReady(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
|
||||
assertResourceEventNotReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
|
||||
readyCh1 := assertResourceEventUntilReady(t, w, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
|
||||
readyCh2 := assertResourceEventUntilReady(t, w, message.NewResourceKeyAckAllBroadcastEvent(message.NewCollectionNameResourceKey("c3")))
|
||||
ack(bc, 7, "v1")
|
||||
<-readyCh1
|
||||
select {
|
||||
case <-readyCh2:
|
||||
assert.Fail(t, "should not ready")
|
||||
case <-time.After(20 * time.Millisecond):
|
||||
}
|
||||
ack(bc, 7, "v2")
|
||||
ack(bc, 7, "v3")
|
||||
<-readyCh2
|
||||
|
||||
w2, _ := bc.NewWatcher()
|
||||
w2.Close() // Close by watcher itself.
|
||||
_, ok := <-w2.EventChan()
|
||||
assert.False(t, ok)
|
||||
|
||||
bc.Close()
|
||||
w.Close() // Close by broadcaster.
|
||||
|
||||
result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}))
|
||||
_, err = bc.Broadcast(context.Background(), nil)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, result)
|
||||
err = bc.Ack(context.Background(), types.BroadcastAckRequest{BroadcastID: 3, VChannel: "v1"})
|
||||
err = bc.Ack(context.Background(), types.BroadcastAckRequest{})
|
||||
assert.Error(t, err)
|
||||
ww, err := bc.NewWatcher()
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, ww)
|
||||
}
|
||||
|
||||
func assertResourceEventOK(t *testing.T, w Watcher, ev1 *message.BroadcastEvent) {
|
||||
w.ObserveResourceKeyEvent(context.Background(), ev1)
|
||||
ev2 := <-w.EventChan()
|
||||
assert.True(t, proto.Equal(ev1, ev2))
|
||||
}
|
||||
|
||||
func assertResourceEventNotReady(t *testing.T, w Watcher, ev1 *message.BroadcastEvent) {
|
||||
w.ObserveResourceKeyEvent(context.Background(), ev1)
|
||||
select {
|
||||
case ev2 := <-w.EventChan():
|
||||
t.Errorf("should not receive event, %+v", ev2)
|
||||
case <-time.After(10 * time.Millisecond):
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func assertResourceEventUntilReady(t *testing.T, w Watcher, ev1 *message.BroadcastEvent) <-chan struct{} {
|
||||
w.ObserveResourceKeyEvent(context.Background(), ev1)
|
||||
done := make(chan struct{})
|
||||
go func() {
|
||||
ev2 := <-w.EventChan()
|
||||
assert.True(t, proto.Equal(ev1, ev2))
|
||||
close(done)
|
||||
}()
|
||||
return done
|
||||
}
|
||||
|
||||
func ack(bc Broadcaster, broadcastID uint64, vchannel string) {
|
||||
func ack(broadcaster Broadcaster, broadcastID uint64, vchannel string) {
|
||||
for {
|
||||
if err := bc.Ack(context.Background(), types.BroadcastAckRequest{
|
||||
if err := broadcaster.Ack(context.Background(), types.BroadcastAckRequest{
|
||||
BroadcastID: broadcastID,
|
||||
VChannel: vchannel,
|
||||
}); err == nil {
|
||||
@ -214,15 +130,7 @@ func ack(bc Broadcaster, broadcastID uint64, vchannel string) {
|
||||
}
|
||||
}
|
||||
|
||||
func asyncAck(wg *sync.WaitGroup, bc Broadcaster, broadcastID uint64, vchannel string) {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
ack(bc, broadcastID, vchannel)
|
||||
}()
|
||||
}
|
||||
|
||||
func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int64) {
|
||||
func createOpeartor(t *testing.T, broadcaster *syncutil.Future[Broadcaster]) (*syncutil.Future[AppendOperator], *atomic.Int64) {
|
||||
id := atomic.NewInt64(1)
|
||||
appended := atomic.NewInt64(0)
|
||||
operator := mock_broadcaster.NewMockAppendOperator(t)
|
||||
@ -230,7 +138,7 @@ func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int
|
||||
resps := types.AppendResponses{
|
||||
Responses: make([]types.AppendResponse, len(msgs)),
|
||||
}
|
||||
for idx := range msgs {
|
||||
for idx, msg := range msgs {
|
||||
newID := walimplstest.NewTestMessageID(id.Inc())
|
||||
if rand.Int31n(10) < 3 {
|
||||
resps.Responses[idx] = types.AppendResponse{
|
||||
@ -246,6 +154,13 @@ func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int
|
||||
Error: nil,
|
||||
}
|
||||
appended.Inc()
|
||||
|
||||
broadcastID := msg.BroadcastHeader().BroadcastID
|
||||
vchannel := msg.VChannel()
|
||||
go func() {
|
||||
time.Sleep(time.Duration(rand.Int31n(100)) * time.Millisecond)
|
||||
ack(broadcaster.Get(), broadcastID, vchannel)
|
||||
}()
|
||||
}
|
||||
return resps
|
||||
}
|
||||
|
||||
@ -20,7 +20,6 @@ func newBroadcasterMetrics() *broadcasterMetrics {
|
||||
taskTotal: metrics.StreamingCoordBroadcasterTaskTotal.MustCurryWith(constLabel),
|
||||
resourceKeyTotal: metrics.StreamingCoordResourceKeyTotal.MustCurryWith(constLabel),
|
||||
broadcastDuration: metrics.StreamingCoordBroadcastDurationSeconds.With(constLabel),
|
||||
ackAnyOneDuration: metrics.StreamingCoordBroadcasterAckAnyOneDurationSeconds.With(constLabel),
|
||||
ackAllDuration: metrics.StreamingCoordBroadcasterAckAllDurationSeconds.With(constLabel),
|
||||
}
|
||||
}
|
||||
@ -30,7 +29,6 @@ type broadcasterMetrics struct {
|
||||
taskTotal *prometheus.GaugeVec
|
||||
resourceKeyTotal *prometheus.GaugeVec
|
||||
broadcastDuration prometheus.Observer
|
||||
ackAnyOneDuration prometheus.Observer
|
||||
ackAllDuration prometheus.Observer
|
||||
}
|
||||
|
||||
@ -80,11 +78,6 @@ func (g *taskMetricsGuard) ObserveBroadcastDone() {
|
||||
g.broadcastDuration.Observe(time.Since(g.start).Seconds())
|
||||
}
|
||||
|
||||
// ObserverAckOne observes the ack any one.
|
||||
func (g *taskMetricsGuard) ObserveAckAnyOne() {
|
||||
g.ackAnyOneDuration.Observe(time.Since(g.start).Seconds())
|
||||
}
|
||||
|
||||
// ObserverAckOne observes the ack all.
|
||||
func (g *taskMetricsGuard) ObserveAckAll() {
|
||||
g.ackAllDuration.Observe(time.Since(g.start).Seconds())
|
||||
|
||||
@ -1,103 +0,0 @@
|
||||
package broadcaster
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
)
|
||||
|
||||
// newWatcher creates a new watcher.
|
||||
func newWatcher(broadcaster *broadcasterImpl) *watcherImpl {
|
||||
w := &watcherImpl{
|
||||
watcherBGNotifier: syncutil.NewAsyncTaskNotifier[struct{}](),
|
||||
pendingEvents: make(map[string]*message.BroadcastEvent, 0),
|
||||
broadcasterImpl: broadcaster,
|
||||
version: 0,
|
||||
input: make(chan *message.BroadcastEvent),
|
||||
output: make(chan *message.BroadcastEvent),
|
||||
}
|
||||
go w.execute()
|
||||
return w
|
||||
}
|
||||
|
||||
// watcherImpl implement the Watcher interface.
|
||||
type watcherImpl struct {
|
||||
watcherBGNotifier *syncutil.AsyncTaskNotifier[struct{}]
|
||||
pendingEvents map[string]*message.BroadcastEvent
|
||||
*broadcasterImpl
|
||||
version int
|
||||
input chan *message.BroadcastEvent
|
||||
output chan *message.BroadcastEvent
|
||||
}
|
||||
|
||||
func (w *watcherImpl) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
select {
|
||||
case w.input <- ev:
|
||||
return nil
|
||||
case <-w.backgroundTaskNotifier.Context().Done():
|
||||
return w.backgroundTaskNotifier.Context().Err()
|
||||
case <-w.watcherBGNotifier.Context().Done():
|
||||
return w.watcherBGNotifier.Context().Err()
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
}
|
||||
|
||||
func (w *watcherImpl) EventChan() <-chan *message.BroadcastEvent {
|
||||
return w.output
|
||||
}
|
||||
|
||||
func (w *watcherImpl) execute() {
|
||||
defer func() {
|
||||
close(w.output)
|
||||
w.watcherBGNotifier.Finish(struct{}{})
|
||||
}()
|
||||
for {
|
||||
ch := w.manager.WatchAtVersion(w.version)
|
||||
select {
|
||||
case <-w.backgroundTaskNotifier.Context().Done():
|
||||
w.Logger().Info("watcher is exit because of broadcaseter is closing", zap.Int("version", w.version))
|
||||
return
|
||||
case <-w.watcherBGNotifier.Context().Done():
|
||||
w.Logger().Info("watcher is exit because of watcher itself is closing", zap.Int("version", w.version))
|
||||
return
|
||||
case <-ch:
|
||||
w.update()
|
||||
case ev := <-w.input:
|
||||
w.pendingEvents[message.UniqueKeyOfBroadcastEvent(ev)] = ev
|
||||
w.update()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (w *watcherImpl) update() {
|
||||
w.version = w.manager.CurrentVersion()
|
||||
newPendingEvent := make(map[string]*message.BroadcastEvent, len(w.pendingEvents))
|
||||
for key, pendingEvent := range w.pendingEvents {
|
||||
switch ev := pendingEvent.Event.(type) {
|
||||
case *messagespb.BroadcastEvent_ResourceKeyAckAll:
|
||||
task, ok := w.manager.GetBroadcastTaskByResourceKey(message.NewResourceKeyFromProto(ev.ResourceKeyAckAll.ResourceKey))
|
||||
if !ok || task.IsAllAcked() {
|
||||
w.output <- pendingEvent
|
||||
continue
|
||||
}
|
||||
case *messagespb.BroadcastEvent_ResourceKeyAckOne:
|
||||
task, ok := w.manager.GetBroadcastTaskByResourceKey(message.NewResourceKeyFromProto(ev.ResourceKeyAckOne.ResourceKey))
|
||||
if !ok || task.IsAcked() {
|
||||
w.output <- pendingEvent
|
||||
continue
|
||||
}
|
||||
}
|
||||
newPendingEvent[key] = pendingEvent
|
||||
}
|
||||
w.pendingEvents = newPendingEvent
|
||||
}
|
||||
|
||||
func (w *watcherImpl) Close() {
|
||||
w.watcherBGNotifier.Cancel()
|
||||
w.watcherBGNotifier.BlockUntilFinish()
|
||||
}
|
||||
@ -4,7 +4,6 @@ import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/service/broadcast"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
@ -62,18 +61,3 @@ func (s *broadcastServceImpl) Ack(ctx context.Context, req *streamingpb.Broadcas
|
||||
}
|
||||
return &streamingpb.BroadcastAckResponse{}, nil
|
||||
}
|
||||
|
||||
func (s *broadcastServceImpl) Watch(svr streamingpb.StreamingCoordBroadcastService_WatchServer) error {
|
||||
broadcaster, err := s.broadcaster.GetWithContext(svr.Context())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
watcher, err := broadcaster.NewWatcher()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer watcher.Close()
|
||||
|
||||
server := broadcast.NewBroadcastWatchServer(watcher, svr)
|
||||
return server.Execute()
|
||||
}
|
||||
|
||||
@ -1,30 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
type broadcastWatchGrpcServerHelper struct {
|
||||
streamingpb.StreamingCoordBroadcastService_WatchServer
|
||||
}
|
||||
|
||||
// SendResourceKeyEvent sends the resource key event to client.
|
||||
func (h *broadcastWatchGrpcServerHelper) SendResourceKeyEvent(ev *message.BroadcastEvent) error {
|
||||
return h.Send(&streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_EventDone{
|
||||
EventDone: &streamingpb.BroadcastEventWatchResponse{
|
||||
Event: ev,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
// SendCloseResponse sends the close response to client.
|
||||
func (h *broadcastWatchGrpcServerHelper) SendCloseResponse() error {
|
||||
return h.Send(&streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_Close{
|
||||
Close: &streamingpb.CloseBroadcastWatchResponse{},
|
||||
},
|
||||
})
|
||||
}
|
||||
@ -1,117 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
var errClosedByUser = errors.New("closed by user")
|
||||
|
||||
func NewBroadcastWatchServer(
|
||||
w broadcaster.Watcher,
|
||||
streamServer streamingpb.StreamingCoordBroadcastService_WatchServer,
|
||||
) *BroadcastWatchServer {
|
||||
ctx, cancel := context.WithCancelCause(streamServer.Context())
|
||||
s := &BroadcastWatchServer{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
w: w,
|
||||
streamServer: broadcastWatchGrpcServerHelper{
|
||||
streamServer,
|
||||
},
|
||||
}
|
||||
s.SetLogger(resource.Resource().Logger().With(log.FieldComponent("broadcast-watch-server")))
|
||||
return s
|
||||
}
|
||||
|
||||
type BroadcastWatchServer struct {
|
||||
log.Binder
|
||||
ctx context.Context
|
||||
cancel context.CancelCauseFunc
|
||||
w broadcaster.Watcher
|
||||
streamServer broadcastWatchGrpcServerHelper
|
||||
}
|
||||
|
||||
func (s *BroadcastWatchServer) Execute() error {
|
||||
// Start a recv arm to handle the control message from client.
|
||||
go func() {
|
||||
// recv loop will be blocked until the stream is closed.
|
||||
// 1. close by client.
|
||||
// 2. close by server context cancel by return of outside Execute.
|
||||
_ = s.recvLoop()
|
||||
}()
|
||||
|
||||
// Start a send loop on current main goroutine.
|
||||
// the loop will be blocked until:
|
||||
// 1. the stream is broken.
|
||||
// 2. recv arm recv closed and all response is sent.
|
||||
return s.sendLoop()
|
||||
}
|
||||
|
||||
// recvLoop receives the message from client.
|
||||
func (s *BroadcastWatchServer) recvLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
s.cancel(err)
|
||||
s.Logger().Warn("recv arm of stream closed by unexpected error", zap.Error(err))
|
||||
return
|
||||
}
|
||||
s.cancel(errClosedByUser)
|
||||
s.Logger().Info("recv arm of stream closed")
|
||||
}()
|
||||
|
||||
for {
|
||||
req, err := s.streamServer.Recv()
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch req := req.Command.(type) {
|
||||
case *streamingpb.BroadcastWatchRequest_CreateEventWatch:
|
||||
// Add new incoming resource key int watcher.
|
||||
s.w.ObserveResourceKeyEvent(s.streamServer.Context(), req.CreateEventWatch.Event)
|
||||
case *streamingpb.BroadcastWatchRequest_Close:
|
||||
// Ignore the command, the stream will be closed by client with io.EOF
|
||||
default:
|
||||
s.Logger().Warn("unknown command type ignored", zap.Any("command", req))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// sendLoop sends the message to client.
|
||||
func (s *BroadcastWatchServer) sendLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
s.Logger().Warn("send arm of stream closed by unexpected error", zap.Error(err))
|
||||
return
|
||||
}
|
||||
s.Logger().Info("send arm of stream closed")
|
||||
}()
|
||||
for {
|
||||
select {
|
||||
case ev, ok := <-s.w.EventChan():
|
||||
if !ok {
|
||||
return errors.New("watcher is closed")
|
||||
}
|
||||
if err := s.streamServer.SendResourceKeyEvent(ev); err != nil {
|
||||
return err
|
||||
}
|
||||
case <-s.ctx.Done():
|
||||
err := context.Cause(s.ctx)
|
||||
if errors.Is(err, errClosedByUser) {
|
||||
return s.streamServer.SendCloseResponse()
|
||||
}
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1,76 +0,0 @@
|
||||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
"github.com/milvus-io/milvus/pkg/v2/mocks/proto/mock_streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
func TestBroadcastWatch(t *testing.T) {
|
||||
resource.InitForTest()
|
||||
w := mock_broadcaster.NewMockWatcher(t)
|
||||
input := make(chan *message.BroadcastEvent, 5)
|
||||
output := make(chan *message.BroadcastEvent, 5)
|
||||
w.EXPECT().ObserveResourceKeyEvent(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, ev *messagespb.BroadcastEvent) error {
|
||||
output <- ev
|
||||
return nil
|
||||
})
|
||||
w.EXPECT().EventChan().Return(output)
|
||||
streamServer := mock_streamingpb.NewMockStreamingCoordBroadcastService_WatchServer(t)
|
||||
streamServer.EXPECT().Context().Return(context.Background())
|
||||
closed := false
|
||||
streamServer.EXPECT().Recv().RunAndReturn(func() (*streamingpb.BroadcastWatchRequest, error) {
|
||||
if closed {
|
||||
return nil, io.EOF
|
||||
}
|
||||
ev, ok := <-input
|
||||
if !ok {
|
||||
closed = true
|
||||
return &streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_Close{
|
||||
Close: &streamingpb.CloseBroadcastWatchRequest{},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
return &streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_CreateEventWatch{
|
||||
CreateEventWatch: &streamingpb.BroadcastCreateEventWatchRequest{
|
||||
Event: ev,
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
})
|
||||
|
||||
streamOutput := make(chan *message.BroadcastEvent, 5)
|
||||
streamServer.EXPECT().Send(mock.Anything).RunAndReturn(func(bwr *streamingpb.BroadcastWatchResponse) error {
|
||||
if bwr.GetEventDone() != nil {
|
||||
streamOutput <- bwr.GetEventDone().Event
|
||||
}
|
||||
return nil
|
||||
})
|
||||
s := NewBroadcastWatchServer(w, streamServer)
|
||||
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1"))
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c2"))
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3"))
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c4"))
|
||||
done := make(chan struct{})
|
||||
go func() {
|
||||
s.Execute()
|
||||
close(done)
|
||||
}()
|
||||
for i := 0; i < 4; i++ {
|
||||
<-streamOutput
|
||||
}
|
||||
close(input)
|
||||
<-done
|
||||
}
|
||||
@ -128,12 +128,6 @@ var (
|
||||
Buckets: secondsBuckets,
|
||||
})
|
||||
|
||||
StreamingCoordBroadcasterAckAnyOneDurationSeconds = newStreamingCoordHistogramVec(prometheus.HistogramOpts{
|
||||
Name: "broadcaster_ack_any_one_duration_seconds",
|
||||
Help: "Duration of acknowledge any message",
|
||||
Buckets: secondsBuckets,
|
||||
})
|
||||
|
||||
StreamingCoordBroadcasterAckAllDurationSeconds = newStreamingCoordHistogramVec(prometheus.HistogramOpts{
|
||||
Name: "broadcaster_ack_all_duration_seconds",
|
||||
Help: "Duration of acknowledge all message",
|
||||
@ -406,7 +400,6 @@ func registerStreamingCoord(registry *prometheus.Registry) {
|
||||
registry.MustRegister(StreamingCoordAssignmentListenerTotal)
|
||||
registry.MustRegister(StreamingCoordBroadcasterTaskTotal)
|
||||
registry.MustRegister(StreamingCoordBroadcastDurationSeconds)
|
||||
registry.MustRegister(StreamingCoordBroadcasterAckAnyOneDurationSeconds)
|
||||
registry.MustRegister(StreamingCoordBroadcasterAckAllDurationSeconds)
|
||||
registry.MustRegister(StreamingCoordResourceKeyTotal)
|
||||
}
|
||||
|
||||
@ -173,79 +173,6 @@ func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) RunAndReturn(
|
||||
return _c
|
||||
}
|
||||
|
||||
// Watch provides a mock function with given fields: ctx, opts
|
||||
func (_m *MockStreamingCoordBroadcastServiceClient) Watch(ctx context.Context, opts ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Watch")
|
||||
}
|
||||
|
||||
var r0 streamingpb.StreamingCoordBroadcastService_WatchClient
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error)); ok {
|
||||
return rf(ctx, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, ...grpc.CallOption) streamingpb.StreamingCoordBroadcastService_WatchClient); ok {
|
||||
r0 = rf(ctx, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(streamingpb.StreamingCoordBroadcastService_WatchClient)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastServiceClient_Watch_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Watch'
|
||||
type MockStreamingCoordBroadcastServiceClient_Watch_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Watch is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Watch(ctx interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
return &MockStreamingCoordBroadcastServiceClient_Watch_Call{Call: _e.mock.On("Watch",
|
||||
append([]interface{}{ctx}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) Run(run func(ctx context.Context, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-1)
|
||||
for i, a := range args[1:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) Return(_a0 streamingpb.StreamingCoordBroadcastService_WatchClient, _a1 error) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) RunAndReturn(run func(context.Context, ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error)) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockStreamingCoordBroadcastServiceClient creates a new instance of MockStreamingCoordBroadcastServiceClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockStreamingCoordBroadcastServiceClient(t interface {
|
||||
|
||||
@ -1,430 +0,0 @@
|
||||
// Code generated by mockery v2.46.0. DO NOT EDIT.
|
||||
|
||||
package mock_streamingpb
|
||||
|
||||
import (
|
||||
context "context"
|
||||
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
metadata "google.golang.org/grpc/metadata"
|
||||
|
||||
streamingpb "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient is an autogenerated mock type for the StreamingCoordBroadcastService_WatchClient type
|
||||
type MockStreamingCoordBroadcastService_WatchClient struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Expecter struct {
|
||||
mock *mock.Mock
|
||||
}
|
||||
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) EXPECT() *MockStreamingCoordBroadcastService_WatchClient_Expecter {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// CloseSend provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) CloseSend() error {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for CloseSend")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func() error); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CloseSend'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// CloseSend is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) CloseSend() *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call{Call: _e.mock.On("CloseSend")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) RunAndReturn(run func() error) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Context provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Context() context.Context {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Context")
|
||||
}
|
||||
|
||||
var r0 context.Context
|
||||
if rf, ok := ret.Get(0).(func() context.Context); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(context.Context)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Context_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Context is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Context() *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Context_Call{Call: _e.mock.On("Context")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) Return(_a0 context.Context) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) RunAndReturn(run func() context.Context) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Header provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Header() (metadata.MD, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Header")
|
||||
}
|
||||
|
||||
var r0 metadata.MD
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (metadata.MD, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() metadata.MD); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(metadata.MD)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Header_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Header'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Header_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Header is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Header() *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Header_Call{Call: _e.mock.On("Header")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) Return(_a0 metadata.MD, _a1 error) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) RunAndReturn(run func() (metadata.MD, error)) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Recv provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Recv() (*streamingpb.BroadcastWatchResponse, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Recv")
|
||||
}
|
||||
|
||||
var r0 *streamingpb.BroadcastWatchResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (*streamingpb.BroadcastWatchResponse, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() *streamingpb.BroadcastWatchResponse); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*streamingpb.BroadcastWatchResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Recv_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Recv is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Recv() *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Recv_Call{Call: _e.mock.On("Recv")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) Return(_a0 *streamingpb.BroadcastWatchResponse, _a1 error) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) RunAndReturn(run func() (*streamingpb.BroadcastWatchResponse, error)) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// RecvMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) RecvMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for RecvMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// RecvMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) RecvMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Send provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Send(_a0 *streamingpb.BroadcastWatchRequest) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Send")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(*streamingpb.BroadcastWatchRequest) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Send_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Send'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Send_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Send is a helper method to define mock.On call
|
||||
// - _a0 *streamingpb.BroadcastWatchRequest
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Send(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Send_Call{Call: _e.mock.On("Send", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) Run(run func(_a0 *streamingpb.BroadcastWatchRequest)) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*streamingpb.BroadcastWatchRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) RunAndReturn(run func(*streamingpb.BroadcastWatchRequest) error) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SendMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) SendMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SendMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SendMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) SendMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call{Call: _e.mock.On("SendMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Trailer provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Trailer() metadata.MD {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Trailer")
|
||||
}
|
||||
|
||||
var r0 metadata.MD
|
||||
if rf, ok := ret.Get(0).(func() metadata.MD); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(metadata.MD)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Trailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Trailer'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Trailer_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Trailer is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Trailer() *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Trailer_Call{Call: _e.mock.On("Trailer")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) Return(_a0 metadata.MD) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) RunAndReturn(run func() metadata.MD) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockStreamingCoordBroadcastService_WatchClient creates a new instance of MockStreamingCoordBroadcastService_WatchClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockStreamingCoordBroadcastService_WatchClient(t interface {
|
||||
mock.TestingT
|
||||
Cleanup(func())
|
||||
}) *MockStreamingCoordBroadcastService_WatchClient {
|
||||
mock := &MockStreamingCoordBroadcastService_WatchClient{}
|
||||
mock.Mock.Test(t)
|
||||
|
||||
t.Cleanup(func() { mock.AssertExpectations(t) })
|
||||
|
||||
return mock
|
||||
}
|
||||
@ -1,406 +0,0 @@
|
||||
// Code generated by mockery v2.46.0. DO NOT EDIT.
|
||||
|
||||
package mock_streamingpb
|
||||
|
||||
import (
|
||||
context "context"
|
||||
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
metadata "google.golang.org/grpc/metadata"
|
||||
|
||||
streamingpb "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer is an autogenerated mock type for the StreamingCoordBroadcastService_WatchServer type
|
||||
type MockStreamingCoordBroadcastService_WatchServer struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Expecter struct {
|
||||
mock *mock.Mock
|
||||
}
|
||||
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) EXPECT() *MockStreamingCoordBroadcastService_WatchServer_Expecter {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// Context provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) Context() context.Context {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Context")
|
||||
}
|
||||
|
||||
var r0 context.Context
|
||||
if rf, ok := ret.Get(0).(func() context.Context); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(context.Context)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Context_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Context is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Context() *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Context_Call{Call: _e.mock.On("Context")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) Return(_a0 context.Context) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) RunAndReturn(run func() context.Context) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Recv provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) Recv() (*streamingpb.BroadcastWatchRequest, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Recv")
|
||||
}
|
||||
|
||||
var r0 *streamingpb.BroadcastWatchRequest
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (*streamingpb.BroadcastWatchRequest, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() *streamingpb.BroadcastWatchRequest); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*streamingpb.BroadcastWatchRequest)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Recv_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Recv is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Recv() *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Recv_Call{Call: _e.mock.On("Recv")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) Return(_a0 *streamingpb.BroadcastWatchRequest, _a1 error) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) RunAndReturn(run func() (*streamingpb.BroadcastWatchRequest, error)) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// RecvMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) RecvMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for RecvMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// RecvMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) RecvMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Send provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) Send(_a0 *streamingpb.BroadcastWatchResponse) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Send")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(*streamingpb.BroadcastWatchResponse) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_Send_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Send'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Send_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Send is a helper method to define mock.On call
|
||||
// - _a0 *streamingpb.BroadcastWatchResponse
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Send(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Send_Call{Call: _e.mock.On("Send", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) Run(run func(_a0 *streamingpb.BroadcastWatchResponse)) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*streamingpb.BroadcastWatchResponse))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) RunAndReturn(run func(*streamingpb.BroadcastWatchResponse) error) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SendHeader provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SendHeader(_a0 metadata.MD) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SendHeader")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(metadata.MD) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendHeader'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SendHeader is a helper method to define mock.On call
|
||||
// - _a0 metadata.MD
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SendHeader(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call{Call: _e.mock.On("SendHeader", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(metadata.MD))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) RunAndReturn(run func(metadata.MD) error) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SendMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SendMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SendMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SendMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SendMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call{Call: _e.mock.On("SendMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetHeader provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SetHeader(_a0 metadata.MD) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SetHeader")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(metadata.MD) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetHeader'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SetHeader is a helper method to define mock.On call
|
||||
// - _a0 metadata.MD
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SetHeader(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call{Call: _e.mock.On("SetHeader", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(metadata.MD))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) RunAndReturn(run func(metadata.MD) error) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetTrailer provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SetTrailer(_a0 metadata.MD) {
|
||||
_m.Called(_a0)
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetTrailer'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SetTrailer is a helper method to define mock.On call
|
||||
// - _a0 metadata.MD
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SetTrailer(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call{Call: _e.mock.On("SetTrailer", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(metadata.MD))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) Return() *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) RunAndReturn(run func(metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockStreamingCoordBroadcastService_WatchServer creates a new instance of MockStreamingCoordBroadcastService_WatchServer. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockStreamingCoordBroadcastService_WatchServer(t interface {
|
||||
mock.TestingT
|
||||
Cleanup(func())
|
||||
}) *MockStreamingCoordBroadcastService_WatchServer {
|
||||
mock := &MockStreamingCoordBroadcastService_WatchServer{}
|
||||
mock.Mock.Test(t)
|
||||
|
||||
t.Cleanup(func() { mock.AssertExpectations(t) })
|
||||
|
||||
return mock
|
||||
}
|
||||
@ -39,6 +39,11 @@ func (i *ImportMsg) SetID(id UniqueID) {
|
||||
i.Base.MsgID = id
|
||||
}
|
||||
|
||||
func (i *ImportMsg) SetTs(ts Timestamp) {
|
||||
i.Base.Timestamp = ts
|
||||
i.BaseMsg.SetTs(ts)
|
||||
}
|
||||
|
||||
func (i *ImportMsg) Type() MsgType {
|
||||
return i.Base.MsgType
|
||||
}
|
||||
|
||||
@ -61,4 +61,17 @@ func TestImportMsg(t *testing.T) {
|
||||
assert.EqualValues(t, 1000, newMsg.EndTs())
|
||||
|
||||
assert.True(t, msg.Size() > 0)
|
||||
|
||||
msg.SetTs(3000)
|
||||
assert.EqualValues(t, 3000, msg.BeginTs())
|
||||
assert.EqualValues(t, 3000, msg.EndTs())
|
||||
assert.EqualValues(t, 3000, msg.(*ImportMsg).Base.Timestamp)
|
||||
|
||||
msgBytes, err = msg.Marshal(msg)
|
||||
assert.NoError(t, err)
|
||||
newMsg, err = newMsg.Unmarshal(msgBytes)
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, 3000, newMsg.BeginTs())
|
||||
assert.EqualValues(t, 3000, newMsg.EndTs())
|
||||
assert.EqualValues(t, 3000, newMsg.(*ImportMsg).Base.Timestamp)
|
||||
}
|
||||
|
||||
@ -96,9 +96,6 @@ service StreamingCoordBroadcastService {
|
||||
|
||||
// Ack acknowledge broadcast message is consumed.
|
||||
rpc Ack(BroadcastAckRequest) returns (BroadcastAckResponse) {}
|
||||
|
||||
// Watch watch the broadcast events.
|
||||
rpc Watch(stream BroadcastWatchRequest) returns (stream BroadcastWatchResponse) {}
|
||||
}
|
||||
|
||||
// BroadcastRequest is the request of the Broadcast RPC.
|
||||
@ -120,34 +117,6 @@ message BroadcastAckRequest {
|
||||
message BroadcastAckResponse {
|
||||
}
|
||||
|
||||
message BroadcastWatchRequest {
|
||||
oneof command {
|
||||
BroadcastCreateEventWatchRequest create_event_watch = 1;
|
||||
CloseBroadcastWatchRequest close = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message BroadcastCreateEventWatchRequest {
|
||||
messages.BroadcastEvent event = 1;
|
||||
}
|
||||
|
||||
message CloseBroadcastWatchRequest {
|
||||
}
|
||||
|
||||
message BroadcastWatchResponse {
|
||||
oneof response {
|
||||
BroadcastEventWatchResponse event_done = 1;
|
||||
CloseBroadcastWatchResponse close = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message BroadcastEventWatchResponse {
|
||||
messages.BroadcastEvent event = 1;
|
||||
}
|
||||
|
||||
message CloseBroadcastWatchResponse {
|
||||
}
|
||||
|
||||
//
|
||||
// StreamingCoordAssignmentService
|
||||
//
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@ -110,7 +110,6 @@ var StreamingNodeStateService_ServiceDesc = grpc.ServiceDesc{
|
||||
const (
|
||||
StreamingCoordBroadcastService_Broadcast_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Broadcast"
|
||||
StreamingCoordBroadcastService_Ack_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Ack"
|
||||
StreamingCoordBroadcastService_Watch_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Watch"
|
||||
)
|
||||
|
||||
// StreamingCoordBroadcastServiceClient is the client API for StreamingCoordBroadcastService service.
|
||||
@ -122,8 +121,6 @@ type StreamingCoordBroadcastServiceClient interface {
|
||||
Broadcast(ctx context.Context, in *BroadcastRequest, opts ...grpc.CallOption) (*BroadcastResponse, error)
|
||||
// Ack acknowledge broadcast message is consumed.
|
||||
Ack(ctx context.Context, in *BroadcastAckRequest, opts ...grpc.CallOption) (*BroadcastAckResponse, error)
|
||||
// Watch watch the broadcast events.
|
||||
Watch(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordBroadcastService_WatchClient, error)
|
||||
}
|
||||
|
||||
type streamingCoordBroadcastServiceClient struct {
|
||||
@ -152,37 +149,6 @@ func (c *streamingCoordBroadcastServiceClient) Ack(ctx context.Context, in *Broa
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *streamingCoordBroadcastServiceClient) Watch(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordBroadcastService_WatchClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &StreamingCoordBroadcastService_ServiceDesc.Streams[0], StreamingCoordBroadcastService_Watch_FullMethodName, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &streamingCoordBroadcastServiceWatchClient{stream}
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type StreamingCoordBroadcastService_WatchClient interface {
|
||||
Send(*BroadcastWatchRequest) error
|
||||
Recv() (*BroadcastWatchResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type streamingCoordBroadcastServiceWatchClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchClient) Send(m *BroadcastWatchRequest) error {
|
||||
return x.ClientStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchClient) Recv() (*BroadcastWatchResponse, error) {
|
||||
m := new(BroadcastWatchResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// StreamingCoordBroadcastServiceServer is the server API for StreamingCoordBroadcastService service.
|
||||
// All implementations should embed UnimplementedStreamingCoordBroadcastServiceServer
|
||||
// for forward compatibility
|
||||
@ -192,8 +158,6 @@ type StreamingCoordBroadcastServiceServer interface {
|
||||
Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error)
|
||||
// Ack acknowledge broadcast message is consumed.
|
||||
Ack(context.Context, *BroadcastAckRequest) (*BroadcastAckResponse, error)
|
||||
// Watch watch the broadcast events.
|
||||
Watch(StreamingCoordBroadcastService_WatchServer) error
|
||||
}
|
||||
|
||||
// UnimplementedStreamingCoordBroadcastServiceServer should be embedded to have forward compatible implementations.
|
||||
@ -206,9 +170,6 @@ func (UnimplementedStreamingCoordBroadcastServiceServer) Broadcast(context.Conte
|
||||
func (UnimplementedStreamingCoordBroadcastServiceServer) Ack(context.Context, *BroadcastAckRequest) (*BroadcastAckResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Ack not implemented")
|
||||
}
|
||||
func (UnimplementedStreamingCoordBroadcastServiceServer) Watch(StreamingCoordBroadcastService_WatchServer) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Watch not implemented")
|
||||
}
|
||||
|
||||
// UnsafeStreamingCoordBroadcastServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to StreamingCoordBroadcastServiceServer will
|
||||
@ -257,32 +218,6 @@ func _StreamingCoordBroadcastService_Ack_Handler(srv interface{}, ctx context.Co
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _StreamingCoordBroadcastService_Watch_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
return srv.(StreamingCoordBroadcastServiceServer).Watch(&streamingCoordBroadcastServiceWatchServer{stream})
|
||||
}
|
||||
|
||||
type StreamingCoordBroadcastService_WatchServer interface {
|
||||
Send(*BroadcastWatchResponse) error
|
||||
Recv() (*BroadcastWatchRequest, error)
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type streamingCoordBroadcastServiceWatchServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchServer) Send(m *BroadcastWatchResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchServer) Recv() (*BroadcastWatchRequest, error) {
|
||||
m := new(BroadcastWatchRequest)
|
||||
if err := x.ServerStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// StreamingCoordBroadcastService_ServiceDesc is the grpc.ServiceDesc for StreamingCoordBroadcastService service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
@ -299,14 +234,7 @@ var StreamingCoordBroadcastService_ServiceDesc = grpc.ServiceDesc{
|
||||
Handler: _StreamingCoordBroadcastService_Ack_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{
|
||||
{
|
||||
StreamName: "Watch",
|
||||
Handler: _StreamingCoordBroadcastService_Watch_Handler,
|
||||
ServerStreams: true,
|
||||
ClientStreams: true,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Metadata: "streaming.proto",
|
||||
}
|
||||
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user