enhance: remove redundant resource key watch operation, just keep consistency of wal (#40503)

issue: #38399
pr: #40235

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.
- Add metrics for broadcast.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
pull/40728/head
Zhen Ye 2025-03-18 15:46:25 +08:00 committed by GitHub
parent d9635c6f86
commit 167d058682
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
38 changed files with 1198 additions and 4382 deletions

View File

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

View File

@ -115,20 +115,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.

View File

@ -15,7 +15,6 @@ import (
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
"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"
)
@ -39,9 +38,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,
@ -68,53 +64,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) {

View File

@ -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()
resp = w.AppendMessages(ctx, newInsertMessage(vChannel1))
@ -156,12 +149,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 {

View File

@ -1,163 +0,0 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_broadcast
import (
context "context"
messagespb "github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
mock "github.com/stretchr/testify/mock"
)
// MockWatcher is an autogenerated mock type for the Watcher type
type MockWatcher struct {
mock.Mock
}
type MockWatcher_Expecter struct {
mock *mock.Mock
}
func (_m *MockWatcher) EXPECT() *MockWatcher_Expecter {
return &MockWatcher_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with given fields:
func (_m *MockWatcher) Close() {
_m.Called()
}
// MockWatcher_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockWatcher_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) Close() *MockWatcher_Close_Call {
return &MockWatcher_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockWatcher_Close_Call) Run(run func()) *MockWatcher_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_Close_Call) Return() *MockWatcher_Close_Call {
_c.Call.Return()
return _c
}
func (_c *MockWatcher_Close_Call) RunAndReturn(run func()) *MockWatcher_Close_Call {
_c.Call.Return(run)
return _c
}
// EventChan provides a mock function with given fields:
func (_m *MockWatcher) EventChan() <-chan *messagespb.BroadcastEvent {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for EventChan")
}
var r0 <-chan *messagespb.BroadcastEvent
if rf, ok := ret.Get(0).(func() <-chan *messagespb.BroadcastEvent); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan *messagespb.BroadcastEvent)
}
}
return r0
}
// MockWatcher_EventChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'EventChan'
type MockWatcher_EventChan_Call struct {
*mock.Call
}
// EventChan is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) EventChan() *MockWatcher_EventChan_Call {
return &MockWatcher_EventChan_Call{Call: _e.mock.On("EventChan")}
}
func (_c *MockWatcher_EventChan_Call) Run(run func()) *MockWatcher_EventChan_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_EventChan_Call) Return(_a0 <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_EventChan_Call) RunAndReturn(run func() <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(run)
return _c
}
// ObserveResourceKeyEvent provides a mock function with given fields: ctx, ev
func (_m *MockWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error {
ret := _m.Called(ctx, ev)
if len(ret) == 0 {
panic("no return value specified for ObserveResourceKeyEvent")
}
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
}
// MockWatcher_ObserveResourceKeyEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ObserveResourceKeyEvent'
type MockWatcher_ObserveResourceKeyEvent_Call struct {
*mock.Call
}
// ObserveResourceKeyEvent is a helper method to define mock.On call
// - ctx context.Context
// - ev *messagespb.BroadcastEvent
func (_e *MockWatcher_Expecter) ObserveResourceKeyEvent(ctx interface{}, ev interface{}) *MockWatcher_ObserveResourceKeyEvent_Call {
return &MockWatcher_ObserveResourceKeyEvent_Call{Call: _e.mock.On("ObserveResourceKeyEvent", ctx, ev)}
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent))
})
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Return(_a0 error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(run)
return _c
}
// NewMockWatcher creates a new instance of MockWatcher. 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 NewMockWatcher(t interface {
mock.TestingT
Cleanup(func())
}) *MockWatcher {
mock := &MockWatcher{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

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

View File

@ -1,163 +0,0 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_broadcaster
import (
context "context"
messagespb "github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
mock "github.com/stretchr/testify/mock"
)
// MockWatcher is an autogenerated mock type for the Watcher type
type MockWatcher struct {
mock.Mock
}
type MockWatcher_Expecter struct {
mock *mock.Mock
}
func (_m *MockWatcher) EXPECT() *MockWatcher_Expecter {
return &MockWatcher_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with given fields:
func (_m *MockWatcher) Close() {
_m.Called()
}
// MockWatcher_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockWatcher_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) Close() *MockWatcher_Close_Call {
return &MockWatcher_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockWatcher_Close_Call) Run(run func()) *MockWatcher_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_Close_Call) Return() *MockWatcher_Close_Call {
_c.Call.Return()
return _c
}
func (_c *MockWatcher_Close_Call) RunAndReturn(run func()) *MockWatcher_Close_Call {
_c.Call.Return(run)
return _c
}
// EventChan provides a mock function with given fields:
func (_m *MockWatcher) EventChan() <-chan *messagespb.BroadcastEvent {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for EventChan")
}
var r0 <-chan *messagespb.BroadcastEvent
if rf, ok := ret.Get(0).(func() <-chan *messagespb.BroadcastEvent); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan *messagespb.BroadcastEvent)
}
}
return r0
}
// MockWatcher_EventChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'EventChan'
type MockWatcher_EventChan_Call struct {
*mock.Call
}
// EventChan is a helper method to define mock.On call
func (_e *MockWatcher_Expecter) EventChan() *MockWatcher_EventChan_Call {
return &MockWatcher_EventChan_Call{Call: _e.mock.On("EventChan")}
}
func (_c *MockWatcher_EventChan_Call) Run(run func()) *MockWatcher_EventChan_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWatcher_EventChan_Call) Return(_a0 <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_EventChan_Call) RunAndReturn(run func() <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call {
_c.Call.Return(run)
return _c
}
// ObserveResourceKeyEvent provides a mock function with given fields: ctx, ev
func (_m *MockWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error {
ret := _m.Called(ctx, ev)
if len(ret) == 0 {
panic("no return value specified for ObserveResourceKeyEvent")
}
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
}
// MockWatcher_ObserveResourceKeyEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ObserveResourceKeyEvent'
type MockWatcher_ObserveResourceKeyEvent_Call struct {
*mock.Call
}
// ObserveResourceKeyEvent is a helper method to define mock.On call
// - ctx context.Context
// - ev *messagespb.BroadcastEvent
func (_e *MockWatcher_Expecter) ObserveResourceKeyEvent(ctx interface{}, ev interface{}) *MockWatcher_ObserveResourceKeyEvent_Call {
return &MockWatcher_ObserveResourceKeyEvent_Call{Call: _e.mock.On("ObserveResourceKeyEvent", ctx, ev)}
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent))
})
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Return(_a0 error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWatcher_ObserveResourceKeyEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockWatcher_ObserveResourceKeyEvent_Call {
_c.Call.Return(run)
return _c
}
// NewMockWatcher creates a new instance of MockWatcher. 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 NewMockWatcher(t interface {
mock.TestingT
Cleanup(func())
}) *MockWatcher {
mock := &MockWatcher{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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"
@ -19,9 +18,11 @@ import (
// newBroadcastTaskManager creates a new broadcast task manager with recovery info.
func newBroadcastTaskManager(protos []*streamingpb.BroadcastTask) (*broadcastTaskManager, []*pendingBroadcastTask) {
logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster"))
metrics := newBroadcasterMetrics()
recoveryTasks := make([]*broadcastTask, 0, len(protos))
for _, proto := range protos {
t := newBroadcastTaskFromProto(proto)
t := newBroadcastTaskFromProto(proto, metrics)
t.SetLogger(logger.With(zap.Uint64("broadcastID", t.header.BroadcastID)))
recoveryTasks = append(recoveryTasks, t)
}
@ -34,6 +35,7 @@ func newBroadcastTaskManager(protos []*streamingpb.BroadcastTask) (*broadcastTas
panic(fmt.Sprintf("unreachable: dirty recovery info in metastore, broadcast ids: [%d, %d]", oldTaskID, task.header.BroadcastID))
}
rks[rk] = task.header.BroadcastID
metrics.IncomingResourceKey(rk.Domain)
}
tasks[task.header.BroadcastID] = task
if task.task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING {
@ -46,7 +48,7 @@ func newBroadcastTaskManager(protos []*streamingpb.BroadcastTask) (*broadcastTas
cond: syncutil.NewContextCond(&sync.Mutex{}),
tasks: tasks,
resourceKeys: rks,
version: 1,
metrics: metrics,
}
m.SetLogger(logger)
return m, pendingTasks
@ -58,22 +60,30 @@ type broadcastTaskManager struct {
cond *syncutil.ContextCond
tasks map[uint64]*broadcastTask // map the broadcastID to the broadcastTaskState
resourceKeys map[message.ResourceKey]uint64 // map the resource key to the broadcastID
version int // version is used to make sure that there's no update lost for watcher.
metrics *broadcasterMetrics
}
// AddTask adds a new broadcast task into the manager.
func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.BroadcastMutableMessage) (*pendingBroadcastTask, error) {
var err error
if msg, err = bm.assignID(ctx, msg); err != nil {
return nil, err
}
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) {
id, err := resource.Resource().IDAllocator().Allocate(ctx)
if err != nil {
return nil, errors.Wrapf(err, "allocate new id failed")
}
msg = msg.WithBroadcastID(id)
task, err := bm.addBroadcastTask(msg)
if err != nil {
return nil, err
}
return newPendingBroadcastTask(task), nil
return msg, nil
}
// Ack acknowledges the message at the specified vchannel.
@ -89,71 +99,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) {
newIncomingTask := newBroadcastTaskFromBroadcastMessage(msg)
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
}
@ -163,20 +158,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()
}

View File

@ -16,21 +16,29 @@ import (
)
// newBroadcastTaskFromProto creates a new broadcast task from the proto.
func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask) *broadcastTask {
func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask, metrics *broadcasterMetrics) *broadcastTask {
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) *broadcastTask {
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,
@ -40,7 +48,13 @@ func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage) *
AckedVchannelBitmap: make([]byte, len(header.VChannels)),
},
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.
@ -50,6 +64,8 @@ type broadcastTask struct {
header *message.BroadcastHeader
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.
@ -116,9 +132,23 @@ func (b *broadcastTask) Ack(ctx context.Context, vchannel string) error {
return err
}
b.task = task
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) {
@ -159,6 +189,7 @@ func (b *broadcastTask) BroadcastDone(ctx context.Context) error {
return err
}
b.task = task
b.metrics.ObserveBroadcastDone()
return nil
}
@ -175,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 {
@ -201,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)))
@ -221,5 +233,6 @@ func (b *broadcastTask) saveTask(ctx context.Context, task *streamingpb.Broadcas
return err
}
logger.Info("save broadcast task done")
b.metrics.ToState(task.State)
return nil
}

View File

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

View File

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

View File

@ -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,136 +73,55 @@ 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.
var result *types.BroadcastAppendResult
for {
var err error
result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}, message.NewCollectionNameResourceKey("c7")))
if err == nil {
break
broadcastWithSameRK := func() {
var result *types.BroadcastAppendResult
for {
var err error
result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}, message.NewCollectionNameResourceKey("c7")))
if err == nil {
break
}
}
assert.Equal(t, len(result.AppendResults), 3)
}
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
}

View File

@ -0,0 +1,84 @@
package broadcaster
import (
"time"
"github.com/prometheus/client_golang/prometheus"
"github.com/milvus-io/milvus/pkg/v2/metrics"
"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/util/paramtable"
)
// newBroadcasterMetrics creates a new broadcaster metrics.
func newBroadcasterMetrics() *broadcasterMetrics {
constLabel := prometheus.Labels{
metrics.NodeIDLabelName: paramtable.GetStringNodeID(),
}
return &broadcasterMetrics{
taskTotal: metrics.StreamingCoordBroadcasterTaskTotal.MustCurryWith(constLabel),
resourceKeyTotal: metrics.StreamingCoordResourceKeyTotal.MustCurryWith(constLabel),
broadcastDuration: metrics.StreamingCoordBroadcastDurationSeconds.With(constLabel),
ackAllDuration: metrics.StreamingCoordBroadcasterAckAllDurationSeconds.With(constLabel),
}
}
// broadcasterMetrics is the metrics of the broadcaster.
type broadcasterMetrics struct {
taskTotal *prometheus.GaugeVec
resourceKeyTotal *prometheus.GaugeVec
broadcastDuration prometheus.Observer
ackAllDuration prometheus.Observer
}
// fromStateToState updates the metrics when the state of the broadcast task changes.
func (m *broadcasterMetrics) fromStateToState(from streamingpb.BroadcastTaskState, to streamingpb.BroadcastTaskState) {
if from != streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN {
m.taskTotal.WithLabelValues(from.String()).Dec()
}
if to != streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE {
m.taskTotal.WithLabelValues(to.String()).Inc()
}
}
// NewBroadcastTask creates a new broadcast task.
func (m *broadcasterMetrics) NewBroadcastTask(state streamingpb.BroadcastTaskState) *taskMetricsGuard {
g := &taskMetricsGuard{
start: time.Now(),
state: state,
broadcasterMetrics: m,
}
g.broadcasterMetrics.fromStateToState(streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN, state)
return g
}
func (m *broadcasterMetrics) IncomingResourceKey(domain messagespb.ResourceDomain) {
m.resourceKeyTotal.WithLabelValues(domain.String()).Inc()
}
func (m *broadcasterMetrics) GoneResourceKey(domain messagespb.ResourceDomain) {
m.resourceKeyTotal.WithLabelValues(domain.String()).Dec()
}
type taskMetricsGuard struct {
start time.Time
state streamingpb.BroadcastTaskState
*broadcasterMetrics
}
// ToState updates the state of the broadcast task.
func (g *taskMetricsGuard) ToState(state streamingpb.BroadcastTaskState) {
g.broadcasterMetrics.fromStateToState(g.state, state)
g.state = state
}
// ObserveBroadcastDone observes the broadcast done.
func (g *taskMetricsGuard) ObserveBroadcastDone() {
g.broadcastDuration.Observe(time.Since(g.start).Seconds())
}
// ObserverAckOne observes the ack all.
func (g *taskMetricsGuard) ObserveAckAll() {
g.ackAllDuration.Observe(time.Since(g.start).Seconds())
}

View File

@ -16,7 +16,9 @@ import (
var errBroadcastTaskIsNotDone = errors.New("broadcast task is not done")
// newPendingBroadcastTask creates a new pendingBroadcastTask.
func newPendingBroadcastTask(task *broadcastTask) *pendingBroadcastTask {
func newPendingBroadcastTask(
task *broadcastTask,
) *pendingBroadcastTask {
msgs := task.PendingBroadcastMessages()
return &pendingBroadcastTask{
broadcastTask: task,
@ -40,6 +42,7 @@ type pendingBroadcastTask struct {
pendingMessages []message.MutableMessage
appendResult map[string]*types.AppendResult
future *syncutil.Future[*types.BroadcastAppendResult]
metrics *taskMetricsGuard
*typeutil.BackoffWithInstant
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -17,6 +17,8 @@ const (
StreamingServiceClientStatusCancel = "cancel"
StreamignServiceClientStatusError = "error"
BroadcasterTaskStateLabelName = "state"
ResourceKeyDomainLabelName = "domain"
TimeTickSyncTypeLabelName = "type"
TimeTickAckTypeLabelName = "type"
WALTxnStateLabelName = "state"
@ -97,6 +99,28 @@ var (
Help: "Total of assignment listener",
})
StreamingCoordBroadcasterTaskTotal = newStreamingCoordGaugeVec(prometheus.GaugeOpts{
Name: "broadcaster_task_total",
Help: "Total of broadcaster task",
}, BroadcasterTaskStateLabelName)
StreamingCoordBroadcastDurationSeconds = newStreamingCoordHistogramVec(prometheus.HistogramOpts{
Name: "broadcaster_broadcast_duration_seconds",
Help: "Duration of broadcast",
Buckets: secondsBuckets,
})
StreamingCoordBroadcasterAckAllDurationSeconds = newStreamingCoordHistogramVec(prometheus.HistogramOpts{
Name: "broadcaster_ack_all_duration_seconds",
Help: "Duration of acknowledge all message",
Buckets: secondsBuckets,
})
StreamingCoordResourceKeyTotal = newStreamingCoordGaugeVec(prometheus.GaugeOpts{
Name: "resource_key_total",
Help: "Total of resource key hold at streaming coord",
}, ResourceKeyDomainLabelName)
// StreamingNode Producer Server Metrics.
StreamingNodeProducerTotal = newStreamingNodeGaugeVec(prometheus.GaugeOpts{
Name: "producer_total",
@ -302,6 +326,10 @@ func registerStreamingCoord(registry *prometheus.Registry) {
registry.MustRegister(StreamingCoordPChannelInfo)
registry.MustRegister(StreamingCoordAssignmentVersion)
registry.MustRegister(StreamingCoordAssignmentListenerTotal)
registry.MustRegister(StreamingCoordBroadcasterTaskTotal)
registry.MustRegister(StreamingCoordBroadcastDurationSeconds)
registry.MustRegister(StreamingCoordBroadcasterAckAllDurationSeconds)
registry.MustRegister(StreamingCoordResourceKeyTotal)
}
// RegisterStreamingNode registers streaming node metrics
@ -356,6 +384,13 @@ func newStreamingCoordGaugeVec(opts prometheus.GaugeOpts, extra ...string) *prom
return prometheus.NewGaugeVec(opts, labels)
}
func newStreamingCoordHistogramVec(opts prometheus.HistogramOpts, extra ...string) *prometheus.HistogramVec {
opts.Namespace = milvusNamespace
opts.Subsystem = typeutil.StreamingCoordRole
labels := mergeLabel(extra...)
return prometheus.NewHistogramVec(opts, labels)
}
func newStreamingServiceClientGaugeVec(opts prometheus.GaugeOpts, extra ...string) *prometheus.GaugeVec {
opts.Namespace = milvusNamespace
opts.Subsystem = subsystemStreamingServiceClient

View File

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

View File

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

View File

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

View File

@ -273,21 +273,3 @@ message ResourceKey {
ResourceDomain domain = 1;
string key = 2;
}
// BroadcastEvent is the event of broadcast message.
message BroadcastEvent {
oneof event {
BroadcastResourceKeyAckAll resource_key_ack_all = 1;
BroadcastResourceKeyAckOne resource_key_ack_one = 2;
}
}
// The resource key is acked by all vchannels.
message BroadcastResourceKeyAckAll {
messages.ResourceKey resource_key = 1;
}
// The resource key is acked by any vchannel.
message BroadcastResourceKeyAckOne {
messages.ResourceKey resource_key = 1;
}

View File

@ -1861,184 +1861,6 @@ func (x *ResourceKey) GetKey() string {
return ""
}
// BroadcastEvent is the event of broadcast message.
type BroadcastEvent struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Types that are assignable to Event:
//
// *BroadcastEvent_ResourceKeyAckAll
// *BroadcastEvent_ResourceKeyAckOne
Event isBroadcastEvent_Event `protobuf_oneof:"event"`
}
func (x *BroadcastEvent) Reset() {
*x = BroadcastEvent{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastEvent) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastEvent) ProtoMessage() {}
func (x *BroadcastEvent) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[32]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastEvent.ProtoReflect.Descriptor instead.
func (*BroadcastEvent) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{32}
}
func (m *BroadcastEvent) GetEvent() isBroadcastEvent_Event {
if m != nil {
return m.Event
}
return nil
}
func (x *BroadcastEvent) GetResourceKeyAckAll() *BroadcastResourceKeyAckAll {
if x, ok := x.GetEvent().(*BroadcastEvent_ResourceKeyAckAll); ok {
return x.ResourceKeyAckAll
}
return nil
}
func (x *BroadcastEvent) GetResourceKeyAckOne() *BroadcastResourceKeyAckOne {
if x, ok := x.GetEvent().(*BroadcastEvent_ResourceKeyAckOne); ok {
return x.ResourceKeyAckOne
}
return nil
}
type isBroadcastEvent_Event interface {
isBroadcastEvent_Event()
}
type BroadcastEvent_ResourceKeyAckAll struct {
ResourceKeyAckAll *BroadcastResourceKeyAckAll `protobuf:"bytes,1,opt,name=resource_key_ack_all,json=resourceKeyAckAll,proto3,oneof"`
}
type BroadcastEvent_ResourceKeyAckOne struct {
ResourceKeyAckOne *BroadcastResourceKeyAckOne `protobuf:"bytes,2,opt,name=resource_key_ack_one,json=resourceKeyAckOne,proto3,oneof"`
}
func (*BroadcastEvent_ResourceKeyAckAll) isBroadcastEvent_Event() {}
func (*BroadcastEvent_ResourceKeyAckOne) isBroadcastEvent_Event() {}
// The resource key is acked by all vchannels.
type BroadcastResourceKeyAckAll struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ResourceKey *ResourceKey `protobuf:"bytes,1,opt,name=resource_key,json=resourceKey,proto3" json:"resource_key,omitempty"`
}
func (x *BroadcastResourceKeyAckAll) Reset() {
*x = BroadcastResourceKeyAckAll{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastResourceKeyAckAll) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastResourceKeyAckAll) ProtoMessage() {}
func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastResourceKeyAckAll.ProtoReflect.Descriptor instead.
func (*BroadcastResourceKeyAckAll) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{33}
}
func (x *BroadcastResourceKeyAckAll) GetResourceKey() *ResourceKey {
if x != nil {
return x.ResourceKey
}
return nil
}
// The resource key is acked by any vchannel.
type BroadcastResourceKeyAckOne struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ResourceKey *ResourceKey `protobuf:"bytes,1,opt,name=resource_key,json=resourceKey,proto3" json:"resource_key,omitempty"`
}
func (x *BroadcastResourceKeyAckOne) Reset() {
*x = BroadcastResourceKeyAckOne{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *BroadcastResourceKeyAckOne) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*BroadcastResourceKeyAckOne) ProtoMessage() {}
func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[34]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use BroadcastResourceKeyAckOne.ProtoReflect.Descriptor instead.
func (*BroadcastResourceKeyAckOne) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{34}
}
func (x *BroadcastResourceKeyAckOne) GetResourceKey() *ResourceKey {
if x != nil {
return x.ResourceKey
}
return nil
}
var File_messages_proto protoreflect.FileDescriptor
var file_messages_proto_rawDesc = []byte{
@ -2214,69 +2036,42 @@ var file_messages_proto_rawDesc = []byte{
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06,
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20,
0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f,
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f,
0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c,
0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65,
0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73,
0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f,
0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74,
0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45,
0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b,
0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f,
0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x2a, 0xfc, 0x01, 0x0a, 0x0b, 0x4d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e,
0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54,
0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10,
0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a,
0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61,
0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12,
0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74,
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61,
0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43,
0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0d,
0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a,
0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a,
0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12,
0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78,
0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b,
0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67,
0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69,
0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f,
0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f,
0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54,
0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c,
0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61,
0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70,
0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33,
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f,
0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x2a, 0xfc, 0x01, 0x0a, 0x0b, 0x4d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e,
0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63,
0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12,
0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46,
0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65,
0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e,
0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06,
0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72,
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75,
0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65,
0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0d, 0x0a, 0x08,
0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43,
0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52,
0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a,
0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53,
0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69, 0x6e,
0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68,
0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d,
0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69,
0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52,
0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e,
0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, 0x0a, 0x0e,
0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x19,
0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d,
0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32,
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70,
0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@ -2292,7 +2087,7 @@ func file_messages_proto_rawDescGZIP() []byte {
}
var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 38)
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 35)
var file_messages_proto_goTypes = []interface{}{
(MessageType)(0), // 0: milvus.proto.messages.MessageType
(TxnState)(0), // 1: milvus.proto.messages.TxnState
@ -2329,33 +2124,26 @@ var file_messages_proto_goTypes = []interface{}{
(*RMQMessageLayout)(nil), // 32: milvus.proto.messages.RMQMessageLayout
(*BroadcastHeader)(nil), // 33: milvus.proto.messages.BroadcastHeader
(*ResourceKey)(nil), // 34: milvus.proto.messages.ResourceKey
(*BroadcastEvent)(nil), // 35: milvus.proto.messages.BroadcastEvent
(*BroadcastResourceKeyAckAll)(nil), // 36: milvus.proto.messages.BroadcastResourceKeyAckAll
(*BroadcastResourceKeyAckOne)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckOne
nil, // 38: milvus.proto.messages.Message.PropertiesEntry
nil, // 39: milvus.proto.messages.ImmutableMessage.PropertiesEntry
nil, // 40: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
nil, // 35: milvus.proto.messages.Message.PropertiesEntry
nil, // 36: milvus.proto.messages.ImmutableMessage.PropertiesEntry
nil, // 37: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
}
var file_messages_proto_depIdxs = []int32{
38, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
35, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID
39, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
36, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo
4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message
16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment
17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment
40, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
37, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
34, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey
2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain
36, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll
37, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne
34, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey
34, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey
14, // [14:14] is the sub-list for method output_type
14, // [14:14] is the sub-list for method input_type
14, // [14:14] is the sub-list for extension type_name
14, // [14:14] is the sub-list for extension extendee
0, // [0:14] is the sub-list for field type_name
10, // [10:10] is the sub-list for method output_type
10, // [10:10] is the sub-list for method input_type
10, // [10:10] is the sub-list for extension type_name
10, // [10:10] is the sub-list for extension extendee
0, // [0:10] is the sub-list for field type_name
}
func init() { file_messages_proto_init() }
@ -2748,46 +2536,6 @@ func file_messages_proto_init() {
return nil
}
}
file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastEvent); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckAll); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckOne); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
}
file_messages_proto_msgTypes[32].OneofWrappers = []interface{}{
(*BroadcastEvent_ResourceKeyAckAll)(nil),
(*BroadcastEvent_ResourceKeyAckOne)(nil),
}
type x struct{}
out := protoimpl.TypeBuilder{
@ -2795,7 +2543,7 @@ func file_messages_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_messages_proto_rawDesc,
NumEnums: 3,
NumMessages: 38,
NumMessages: 35,
NumExtensions: 0,
NumServices: 0,
},

View File

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

View File

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

View File

@ -3,8 +3,6 @@ package message
import (
"strconv"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
@ -76,36 +74,3 @@ func NewCollectionNameResourceKey(collectionName string) ResourceKey {
Key: collectionName,
}
}
type BroadcastEvent = messagespb.BroadcastEvent
// UniqueKeyOfBroadcastEvent returns a unique key for a broadcast event.
func UniqueKeyOfBroadcastEvent(ev *BroadcastEvent) string {
s, err := proto.Marshal(ev)
if err != nil {
panic(err)
}
return string(s)
}
// NewResourceKeyAckOneBroadcastEvent creates a broadcast event for acking one key.
func NewResourceKeyAckOneBroadcastEvent(rk ResourceKey) *BroadcastEvent {
return &BroadcastEvent{
Event: &messagespb.BroadcastEvent_ResourceKeyAckOne{
ResourceKeyAckOne: &messagespb.BroadcastResourceKeyAckOne{
ResourceKey: rk.IntoResourceKey(),
},
},
}
}
// NewResourceKeyAckAllBroadcastEvent creates a broadcast event for ack all vchannel.
func NewResourceKeyAckAllBroadcastEvent(rk ResourceKey) *BroadcastEvent {
return &BroadcastEvent{
Event: &messagespb.BroadcastEvent_ResourceKeyAckAll{
ResourceKeyAckAll: &messagespb.BroadcastResourceKeyAckAll{
ResourceKey: rk.IntoResourceKey(),
},
},
}
}