mirror of https://github.com/milvus-io/milvus.git
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
parent
d9635c6f86
commit
167d058682
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -32,6 +32,7 @@ func (b *broadcastTask) Execute(ctx context.Context) error {
|
|||
|
||||
for idx, msg := range b.msgs {
|
||||
tsMsg, err := adaptor.NewMsgPackFromMutableMessageV1(msg)
|
||||
tsMsg.SetTs(b.ts) // overwrite the ts.
|
||||
if err != nil {
|
||||
result.FillResponseAtIdx(types.AppendResponse{Error: err}, idx)
|
||||
return err
|
||||
|
|
|
@ -2,35 +2,19 @@ package broadcast
|
|||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
var logger = log.With(log.FieldComponent("broadcast-client"))
|
||||
|
||||
// NewGRPCBroadcastService creates a new broadcast service with grpc.
|
||||
func NewGRPCBroadcastService(walName string, service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]) *GRPCBroadcastServiceImpl {
|
||||
rw := newResumingWatcher(&grpcWatcherBuilder{
|
||||
broadcastService: service,
|
||||
}, &typeutil.BackoffTimerConfig{
|
||||
Default: 5 * time.Second,
|
||||
Backoff: typeutil.BackoffConfig{
|
||||
InitialInterval: 50 * time.Millisecond,
|
||||
Multiplier: 2.0,
|
||||
MaxInterval: 5 * time.Second,
|
||||
},
|
||||
})
|
||||
return &GRPCBroadcastServiceImpl{
|
||||
walName: walName,
|
||||
service: service,
|
||||
w: rw,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -39,7 +23,6 @@ func NewGRPCBroadcastService(walName string, service lazygrpc.Service[streamingp
|
|||
type GRPCBroadcastServiceImpl struct {
|
||||
walName string
|
||||
service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]
|
||||
w *resumingWatcher
|
||||
}
|
||||
|
||||
func (c *GRPCBroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
|
||||
|
@ -87,10 +70,5 @@ func (c *GRPCBroadcastServiceImpl) Ack(ctx context.Context, req types.BroadcastA
|
|||
return err
|
||||
}
|
||||
|
||||
func (c *GRPCBroadcastServiceImpl) BlockUntilEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
return c.w.ObserveResourceKeyEvent(ctx, ev)
|
||||
}
|
||||
|
||||
func (c *GRPCBroadcastServiceImpl) Close() {
|
||||
c.w.Close()
|
||||
}
|
||||
|
|
|
@ -3,10 +3,17 @@ package broadcast
|
|||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/mocks/proto/mock_streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
||||
|
@ -27,7 +34,23 @@ func TestBroadcast(t *testing.T) {
|
|||
BroadcastID: 1,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
err = bs.BlockUntilEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("r1")))
|
||||
assert.NoError(t, err)
|
||||
bs.Close()
|
||||
}
|
||||
|
||||
func newMockServer(t *testing.T, sendDelay time.Duration) lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] {
|
||||
s := mock_lazygrpc.NewMockService[streamingpb.StreamingCoordBroadcastServiceClient](t)
|
||||
c := mock_streamingpb.NewMockStreamingCoordBroadcastServiceClient(t)
|
||||
s.EXPECT().GetService(mock.Anything).Return(c, nil)
|
||||
c.EXPECT().Broadcast(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastResponse{
|
||||
Results: map[string]*streamingpb.ProduceMessageResponseResult{
|
||||
"v1": {
|
||||
Id: &messagespb.MessageID{
|
||||
Id: walimplstest.NewTestMessageID(1).Marshal(),
|
||||
},
|
||||
},
|
||||
},
|
||||
BroadcastId: 1,
|
||||
}, nil).Maybe()
|
||||
c.EXPECT().Ack(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastAckResponse{}, nil).Maybe()
|
||||
return s
|
||||
}
|
||||
|
|
|
@ -1,192 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type grpcWatcherBuilder struct {
|
||||
broadcastService lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]
|
||||
}
|
||||
|
||||
func (b *grpcWatcherBuilder) Build(ctx context.Context) (Watcher, error) {
|
||||
service, err := b.broadcastService.GetService(ctx)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "failed to get broadcast service")
|
||||
}
|
||||
bt := syncutil.NewAsyncTaskNotifier[struct{}]()
|
||||
// TODO: Here we make a broken stream by passing a context.
|
||||
// Implement a graceful closing should be better.
|
||||
streamCtx, cancel := context.WithCancel(context.Background())
|
||||
svr, err := service.Watch(streamCtx)
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, errors.Wrap(err, "failed to create broadcast watcher server client")
|
||||
}
|
||||
w := &grpcWatcherClient{
|
||||
lifetime: typeutil.NewLifetime(),
|
||||
backgroundTask: bt,
|
||||
streamServerCancel: cancel,
|
||||
streamClient: svr,
|
||||
input: make(chan *message.BroadcastEvent),
|
||||
output: make(chan *message.BroadcastEvent),
|
||||
sendExitCh: make(chan struct{}),
|
||||
recvExitCh: make(chan struct{}),
|
||||
}
|
||||
w.SetLogger(logger)
|
||||
go w.executeBackgroundTask()
|
||||
return w, nil
|
||||
}
|
||||
|
||||
type grpcWatcherClient struct {
|
||||
log.Binder
|
||||
lifetime *typeutil.Lifetime
|
||||
backgroundTask *syncutil.AsyncTaskNotifier[struct{}]
|
||||
streamServerCancel context.CancelFunc
|
||||
streamClient streamingpb.StreamingCoordBroadcastService_WatchClient
|
||||
input chan *message.BroadcastEvent
|
||||
output chan *message.BroadcastEvent
|
||||
recvExitCh chan struct{}
|
||||
sendExitCh chan struct{}
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
if !c.lifetime.Add(typeutil.LifetimeStateWorking) {
|
||||
return errWatcherClosed
|
||||
}
|
||||
defer c.lifetime.Done()
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-c.backgroundTask.Context().Done():
|
||||
return c.backgroundTask.Context().Err()
|
||||
case c.input <- ev:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) EventChan() <-chan *message.BroadcastEvent {
|
||||
return c.output
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) gracefulClose() error {
|
||||
c.lifetime.SetState(typeutil.LifetimeStateStopped)
|
||||
// cancel the background task and wait for all request to finish
|
||||
c.backgroundTask.Cancel()
|
||||
c.lifetime.Wait()
|
||||
|
||||
select {
|
||||
case <-c.backgroundTask.FinishChan():
|
||||
return nil
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
return context.DeadlineExceeded
|
||||
}
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) Close() {
|
||||
// Try to make a graceful close.
|
||||
if err := c.gracefulClose(); err != nil {
|
||||
c.Logger().Warn("failed to close the broadcast watcher gracefully, a froce closing will be applied", zap.Error(err))
|
||||
}
|
||||
c.streamServerCancel()
|
||||
c.backgroundTask.BlockUntilFinish()
|
||||
}
|
||||
|
||||
func (c *grpcWatcherClient) executeBackgroundTask() {
|
||||
defer func() {
|
||||
close(c.output)
|
||||
c.backgroundTask.Finish(struct{}{})
|
||||
}()
|
||||
|
||||
go c.recvLoop()
|
||||
go c.sendLoop()
|
||||
<-c.recvExitCh
|
||||
<-c.sendExitCh
|
||||
}
|
||||
|
||||
// sendLoop send the incoming event to the remote server.
|
||||
// If the input channel is closed, it will send a close message to the remote server and return.
|
||||
func (c *grpcWatcherClient) sendLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
c.Logger().Warn("send arm of stream closed by unexpected error", zap.Error(err))
|
||||
} else {
|
||||
c.Logger().Info("send arm of stream closed")
|
||||
}
|
||||
if err := c.streamClient.CloseSend(); err != nil {
|
||||
c.Logger().Warn("failed to close send", zap.Error(err))
|
||||
}
|
||||
close(c.sendExitCh)
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-c.backgroundTask.Context().Done():
|
||||
// send close message stop the loop.
|
||||
// then the server will close the recv arm and return io.EOF.
|
||||
// recv arm can be closed after that.
|
||||
return c.streamClient.Send(&streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_Close{
|
||||
Close: &streamingpb.CloseBroadcastWatchRequest{},
|
||||
},
|
||||
})
|
||||
case ev := <-c.input:
|
||||
if err := c.streamClient.Send(&streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_CreateEventWatch{
|
||||
CreateEventWatch: &streamingpb.BroadcastCreateEventWatchRequest{
|
||||
Event: ev,
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// recvLoop receive the event from the remote server.
|
||||
func (c *grpcWatcherClient) recvLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
c.Logger().Warn("recv arm of stream closed by unexpected error", zap.Error(err))
|
||||
} else {
|
||||
c.Logger().Info("recv arm of stream closed")
|
||||
}
|
||||
close(c.recvExitCh)
|
||||
}()
|
||||
|
||||
for {
|
||||
resp, err := c.streamClient.Recv()
|
||||
if errors.Is(err, io.EOF) {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch resp := resp.Response.(type) {
|
||||
case *streamingpb.BroadcastWatchResponse_EventDone:
|
||||
select {
|
||||
case c.output <- resp.EventDone.Event:
|
||||
case <-c.backgroundTask.Context().Done():
|
||||
c.Logger().Info("recv arm close when send event to output channel, skip wait for io.EOF")
|
||||
return nil
|
||||
}
|
||||
case *streamingpb.BroadcastWatchResponse_Close:
|
||||
// nothing to do now, just wait io.EOF.
|
||||
default:
|
||||
c.Logger().Warn("unknown response type", zap.Any("response", resp))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,125 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
|
||||
"github.com/milvus-io/milvus/pkg/v2/mocks/proto/mock_streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
||||
)
|
||||
|
||||
func TestWatcher(t *testing.T) {
|
||||
s := newMockServer(t, 0)
|
||||
|
||||
b := grpcWatcherBuilder{broadcastService: s}
|
||||
w, err := b.Build(context.Background())
|
||||
assert.NoError(t, err)
|
||||
|
||||
done := make(chan struct{})
|
||||
cnt := 0
|
||||
go func() {
|
||||
defer close(done)
|
||||
for range w.EventChan() {
|
||||
cnt++
|
||||
}
|
||||
}()
|
||||
for i := 0; i < 10; i++ {
|
||||
err = w.ObserveResourceKeyEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
w.Close()
|
||||
<-done
|
||||
assert.Equal(t, 10, cnt)
|
||||
err = w.ObserveResourceKeyEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assert.Error(t, err)
|
||||
|
||||
// Test ungraceful close
|
||||
s = newMockServer(t, 10*time.Second)
|
||||
b2 := grpcWatcherBuilder{broadcastService: s}
|
||||
w2, err := b2.Build(context.Background())
|
||||
assert.NoError(t, err)
|
||||
w2.Close()
|
||||
}
|
||||
|
||||
func newMockServer(t *testing.T, sendDelay time.Duration) lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] {
|
||||
s := mock_lazygrpc.NewMockService[streamingpb.StreamingCoordBroadcastServiceClient](t)
|
||||
c := mock_streamingpb.NewMockStreamingCoordBroadcastServiceClient(t)
|
||||
s.EXPECT().GetService(mock.Anything).Return(c, nil)
|
||||
var ctx context.Context
|
||||
cc := mock_streamingpb.NewMockStreamingCoordBroadcastService_WatchClient(t)
|
||||
c.EXPECT().Watch(mock.Anything).RunAndReturn(func(ctx2 context.Context, co ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error) {
|
||||
ctx = ctx2
|
||||
return cc, nil
|
||||
})
|
||||
c.EXPECT().Broadcast(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastResponse{
|
||||
Results: map[string]*streamingpb.ProduceMessageResponseResult{
|
||||
"v1": {
|
||||
Id: &messagespb.MessageID{
|
||||
Id: walimplstest.NewTestMessageID(1).Marshal(),
|
||||
},
|
||||
},
|
||||
},
|
||||
BroadcastId: 1,
|
||||
}, nil).Maybe()
|
||||
c.EXPECT().Ack(mock.Anything, mock.Anything).Return(&streamingpb.BroadcastAckResponse{}, nil).Maybe()
|
||||
|
||||
output := make(chan *streamingpb.BroadcastWatchRequest, 10)
|
||||
cc.EXPECT().Recv().RunAndReturn(func() (*streamingpb.BroadcastWatchResponse, error) {
|
||||
var result *streamingpb.BroadcastWatchRequest
|
||||
var ok bool
|
||||
select {
|
||||
case result, ok = <-output:
|
||||
if !ok {
|
||||
return nil, io.EOF
|
||||
}
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
switch cmd := result.Command.(type) {
|
||||
case *streamingpb.BroadcastWatchRequest_Close:
|
||||
return &streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_Close{Close: &streamingpb.CloseBroadcastWatchResponse{}},
|
||||
}, nil
|
||||
case *streamingpb.BroadcastWatchRequest_CreateEventWatch:
|
||||
return &streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_EventDone{
|
||||
EventDone: &streamingpb.BroadcastEventWatchResponse{
|
||||
Event: cmd.CreateEventWatch.Event,
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
default:
|
||||
panic("unknown command")
|
||||
}
|
||||
})
|
||||
cc.EXPECT().Send(mock.Anything).RunAndReturn(func(bwr *streamingpb.BroadcastWatchRequest) error {
|
||||
select {
|
||||
case <-time.After(sendDelay):
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
select {
|
||||
case output <- bwr:
|
||||
return nil
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
})
|
||||
cc.EXPECT().CloseSend().RunAndReturn(func() error {
|
||||
close(output)
|
||||
return nil
|
||||
})
|
||||
return s
|
||||
}
|
|
@ -1,22 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
type WatcherBuilder interface {
|
||||
Build(ctx context.Context) (Watcher, error)
|
||||
}
|
||||
|
||||
type Watcher interface {
|
||||
// ObserveResourceKeyEvent observes the resource key event.
|
||||
ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error
|
||||
|
||||
// EventChan returns the event channel.
|
||||
EventChan() <-chan *message.BroadcastEvent
|
||||
|
||||
// Close closes the watcher.
|
||||
Close()
|
||||
}
|
|
@ -1,178 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
var errWatcherClosed = errors.New("watcher is closed")
|
||||
|
||||
// newResumingWatcher create a new resuming watcher.
|
||||
func newResumingWatcher(b WatcherBuilder, backoffConfig *typeutil.BackoffTimerConfig) *resumingWatcher {
|
||||
rw := &resumingWatcher{
|
||||
backgroundTask: syncutil.NewAsyncTaskNotifier[struct{}](),
|
||||
input: make(chan *pendingEvent),
|
||||
evs: &pendingEvents{evs: make(map[string]*pendingEvent)},
|
||||
watcherBuilder: b, // TODO: enable local watcher here.
|
||||
}
|
||||
rw.SetLogger(logger)
|
||||
go rw.execute(backoffConfig)
|
||||
return rw
|
||||
}
|
||||
|
||||
// resumingWatcher is a watcher that can resume the watcher when it is unavailable.
|
||||
type resumingWatcher struct {
|
||||
log.Binder
|
||||
backgroundTask *syncutil.AsyncTaskNotifier[struct{}]
|
||||
input chan *pendingEvent
|
||||
evs *pendingEvents
|
||||
watcherBuilder WatcherBuilder
|
||||
}
|
||||
|
||||
// ObserveResourceKeyEvent observes the resource key event.
|
||||
func (r *resumingWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
notifier := make(chan struct{})
|
||||
select {
|
||||
case <-r.backgroundTask.Context().Done():
|
||||
return errWatcherClosed
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case r.input <- &pendingEvent{
|
||||
ev: ev,
|
||||
notifier: []chan<- struct{}{notifier},
|
||||
}:
|
||||
}
|
||||
select {
|
||||
case <-r.backgroundTask.Context().Done():
|
||||
return errWatcherClosed
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
case <-notifier:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (r *resumingWatcher) Close() {
|
||||
r.backgroundTask.Cancel()
|
||||
r.backgroundTask.BlockUntilFinish()
|
||||
}
|
||||
|
||||
func (r *resumingWatcher) execute(backoffConfig *typeutil.BackoffTimerConfig) {
|
||||
backoff := typeutil.NewBackoffTimer(backoffConfig)
|
||||
var nextTimer <-chan time.Time
|
||||
var initialized bool
|
||||
var watcher Watcher
|
||||
defer func() {
|
||||
if watcher != nil {
|
||||
watcher.Close()
|
||||
}
|
||||
r.backgroundTask.Finish(struct{}{})
|
||||
}()
|
||||
|
||||
for {
|
||||
var eventChan <-chan *message.BroadcastEvent
|
||||
if watcher != nil {
|
||||
eventChan = watcher.EventChan()
|
||||
}
|
||||
|
||||
select {
|
||||
case <-r.backgroundTask.Context().Done():
|
||||
return
|
||||
case ev := <-r.input:
|
||||
if !r.evs.AddPendingEvent(ev) && watcher != nil {
|
||||
if err := watcher.ObserveResourceKeyEvent(r.backgroundTask.Context(), ev.ev); err != nil {
|
||||
watcher.Close()
|
||||
watcher = nil
|
||||
}
|
||||
}
|
||||
if !initialized {
|
||||
// try to initialize watcher in next loop.
|
||||
// avoid to make a grpc stream channel if the watch operation is not used.
|
||||
nextTimer = time.After(0)
|
||||
initialized = true
|
||||
}
|
||||
case ev, ok := <-eventChan:
|
||||
if !ok {
|
||||
watcher.Close()
|
||||
watcher = nil
|
||||
break
|
||||
}
|
||||
r.evs.Notify(ev)
|
||||
case <-nextTimer:
|
||||
var err error
|
||||
nextTimer = nil
|
||||
if watcher, err = r.createNewWatcher(); err != nil {
|
||||
r.Logger().Warn("create new watcher failed", zap.Error(err))
|
||||
break
|
||||
}
|
||||
r.Logger().Info("create new watcher successful")
|
||||
backoff.DisableBackoff()
|
||||
}
|
||||
if watcher == nil && nextTimer == nil {
|
||||
backoff.EnableBackoff()
|
||||
var interval time.Duration
|
||||
nextTimer, interval = backoff.NextTimer()
|
||||
r.Logger().Warn("watcher is unavailable, resuming it after interval", zap.Duration("interval", interval))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (r *resumingWatcher) createNewWatcher() (Watcher, error) {
|
||||
watcher, err := r.watcherBuilder.Build(r.backgroundTask.Context())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := r.evs.SendAll(r.backgroundTask.Context(), watcher); err != nil {
|
||||
watcher.Close()
|
||||
return nil, errors.Wrapf(err, "send all pending events to watcher failed")
|
||||
}
|
||||
return watcher, nil
|
||||
}
|
||||
|
||||
type pendingEvents struct {
|
||||
evs map[string]*pendingEvent
|
||||
}
|
||||
|
||||
// AddPendingEvent adds a pending event.
|
||||
// Return true if the event is already in the pending events.
|
||||
func (evs *pendingEvents) AddPendingEvent(ev *pendingEvent) bool {
|
||||
id := message.UniqueKeyOfBroadcastEvent(ev.ev)
|
||||
if existEv, ok := evs.evs[id]; ok {
|
||||
existEv.notifier = append(existEv.notifier, ev.notifier...)
|
||||
return true
|
||||
}
|
||||
evs.evs[id] = ev
|
||||
return false
|
||||
}
|
||||
|
||||
func (evs *pendingEvents) Notify(ev *message.BroadcastEvent) {
|
||||
id := message.UniqueKeyOfBroadcastEvent(ev)
|
||||
if existEv, ok := evs.evs[id]; ok {
|
||||
for _, notifier := range existEv.notifier {
|
||||
close(notifier)
|
||||
}
|
||||
delete(evs.evs, id)
|
||||
}
|
||||
}
|
||||
|
||||
func (evs *pendingEvents) SendAll(ctx context.Context, w Watcher) error {
|
||||
for _, ev := range evs.evs {
|
||||
if err := w.ObserveResourceKeyEvent(ctx, ev.ev); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type pendingEvent struct {
|
||||
ev *message.BroadcastEvent
|
||||
notifier []chan<- struct{}
|
||||
}
|
|
@ -1,101 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"go.uber.org/atomic"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/client/mock_broadcast"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type mockBuilder struct {
|
||||
built func(ctx context.Context) (Watcher, error)
|
||||
}
|
||||
|
||||
func (b *mockBuilder) Build(ctx context.Context) (Watcher, error) {
|
||||
return b.built(ctx)
|
||||
}
|
||||
|
||||
func TestWatcherResuming(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
b := newMockWatcherBuilder(t)
|
||||
rw := newResumingWatcher(b, &typeutil.BackoffTimerConfig{
|
||||
Default: 500 * time.Millisecond,
|
||||
Backoff: typeutil.BackoffConfig{
|
||||
InitialInterval: 10 * time.Millisecond,
|
||||
Multiplier: 2.0,
|
||||
MaxInterval: 500 * time.Millisecond,
|
||||
},
|
||||
})
|
||||
wg := &sync.WaitGroup{}
|
||||
for i := 0; i < 100; i++ {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
id := rand.Int31n(10)
|
||||
rk := message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey(fmt.Sprintf("c%d", id)))
|
||||
err := rw.ObserveResourceKeyEvent(ctx, rk)
|
||||
assert.NoError(t, err)
|
||||
}()
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
rw.Close()
|
||||
err := rw.ObserveResourceKeyEvent(ctx, message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1")))
|
||||
assert.ErrorIs(t, err, errWatcherClosed)
|
||||
}
|
||||
|
||||
func newMockWatcherBuilder(t *testing.T) WatcherBuilder {
|
||||
return &mockBuilder{built: func(ctx context.Context) (Watcher, error) {
|
||||
w := mock_broadcast.NewMockWatcher(t)
|
||||
n := rand.Int31n(10)
|
||||
if n < 3 {
|
||||
return nil, errors.New("err")
|
||||
}
|
||||
|
||||
// ill watcher
|
||||
k := atomic.NewInt32(n)
|
||||
o := rand.Int31n(20) + n
|
||||
mu := sync.Mutex{}
|
||||
closed := false
|
||||
output := make(chan *message.BroadcastEvent, 500)
|
||||
w.EXPECT().ObserveResourceKeyEvent(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, be *messagespb.BroadcastEvent) error {
|
||||
k2 := k.Inc()
|
||||
if k2 >= o {
|
||||
return errors.New("err")
|
||||
}
|
||||
mu.Lock()
|
||||
if closed {
|
||||
return errors.New("closed")
|
||||
}
|
||||
go func() {
|
||||
defer mu.Unlock()
|
||||
time.Sleep(time.Duration(rand.Int31n(5)) * time.Millisecond)
|
||||
output <- be
|
||||
}()
|
||||
return nil
|
||||
}).Maybe()
|
||||
w.EXPECT().EventChan().RunAndReturn(func() <-chan *messagespb.BroadcastEvent {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
if !closed && rand.Int31n(100) < 50 {
|
||||
close(output)
|
||||
closed = true
|
||||
}
|
||||
return output
|
||||
}).Maybe()
|
||||
w.EXPECT().Close().Return()
|
||||
return w, nil
|
||||
}}
|
||||
}
|
|
@ -44,9 +44,6 @@ type BroadcastService interface {
|
|||
// Ack sends a broadcast ack to the streaming service.
|
||||
Ack(ctx context.Context, req types.BroadcastAckRequest) error
|
||||
|
||||
// BlockUntilEvent blocks until the event happens.
|
||||
BlockUntilEvent(ctx context.Context, ev *message.BroadcastEvent) error
|
||||
|
||||
// Close closes the broadcast service.
|
||||
Close()
|
||||
}
|
||||
|
|
|
@ -9,7 +9,6 @@ import (
|
|||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -15,23 +15,8 @@ type Broadcaster interface {
|
|||
// Ack acknowledges the message at the specified vchannel.
|
||||
Ack(ctx context.Context, req types.BroadcastAckRequest) error
|
||||
|
||||
// Watch watches the broadcast event.
|
||||
NewWatcher() (Watcher, error)
|
||||
|
||||
// Close closes the broadcaster.
|
||||
Close()
|
||||
}
|
||||
|
||||
// Watcher is the interface for watching the broadcast event.
|
||||
type Watcher interface {
|
||||
// ObserveResourceKeyEvent observes the resource key event.
|
||||
ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error
|
||||
|
||||
// EventChan returns the event channel.
|
||||
EventChan() <-chan *message.BroadcastEvent
|
||||
|
||||
// Close closes the watcher.
|
||||
Close()
|
||||
}
|
||||
|
||||
type AppendOperator = registry.AppendOperator
|
||||
|
|
|
@ -84,7 +84,16 @@ func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMu
|
|||
|
||||
// Wait both request context and the background task context.
|
||||
ctx, _ = contextutil.MergeContext(ctx, b.backgroundTaskNotifier.Context())
|
||||
return t.BlockUntilTaskDone(ctx)
|
||||
r, err := t.BlockUntilTaskDone(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// wait for all the vchannels acked.
|
||||
if err := t.BlockUntilAllAck(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// Ack acknowledges the message at the specified vchannel.
|
||||
|
@ -97,15 +106,6 @@ func (b *broadcasterImpl) Ack(ctx context.Context, req types.BroadcastAckRequest
|
|||
return b.manager.Ack(ctx, req.BroadcastID, req.VChannel)
|
||||
}
|
||||
|
||||
func (b *broadcasterImpl) NewWatcher() (Watcher, error) {
|
||||
if !b.lifetime.Add(typeutil.LifetimeStateWorking) {
|
||||
return nil, status.NewOnShutdownError("broadcaster is closing")
|
||||
}
|
||||
defer b.lifetime.Done()
|
||||
|
||||
return newWatcher(b), nil
|
||||
}
|
||||
|
||||
func (b *broadcasterImpl) Close() {
|
||||
b.lifetime.SetState(typeutil.LifetimeStateStopped)
|
||||
b.lifetime.Wait()
|
||||
|
@ -215,6 +215,8 @@ func (b *broadcasterImpl) worker(no int, appendOperator AppendOperator) {
|
|||
case b.backoffChan <- task:
|
||||
}
|
||||
}
|
||||
// All message of broadcast task is sent, release the resource keys to let other task with same resource keys to apply operation.
|
||||
b.manager.ReleaseResourceKeys(task.Header().BroadcastID)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -3,7 +3,6 @@ package broadcaster
|
|||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -11,7 +10,6 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"go.uber.org/atomic"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/mocks/mock_metastore"
|
||||
|
@ -19,7 +17,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
internaltypes "github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/idalloc"
|
||||
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
|
@ -37,8 +34,8 @@ func TestBroadcaster(t *testing.T) {
|
|||
meta.EXPECT().ListBroadcastTask(mock.Anything).
|
||||
RunAndReturn(func(ctx context.Context) ([]*streamingpb.BroadcastTask, error) {
|
||||
return []*streamingpb.BroadcastTask{
|
||||
createNewBroadcastTask(1, []string{"v1"}, message.NewCollectionNameResourceKey("c1")),
|
||||
createNewBroadcastTask(2, []string{"v1", "v2"}, message.NewCollectionNameResourceKey("c2")),
|
||||
createNewBroadcastTask(8, []string{"v1"}, message.NewCollectionNameResourceKey("c1")),
|
||||
createNewBroadcastTask(9, []string{"v1", "v2"}, message.NewCollectionNameResourceKey("c2")),
|
||||
createNewBroadcastTask(3, []string{"v1", "v2", "v3"}),
|
||||
createNewWaitAckBroadcastTaskFromMessage(
|
||||
createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(4),
|
||||
|
@ -76,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
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
}
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -1,103 +0,0 @@
|
|||
package broadcaster
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
|
||||
)
|
||||
|
||||
// newWatcher creates a new watcher.
|
||||
func newWatcher(broadcaster *broadcasterImpl) *watcherImpl {
|
||||
w := &watcherImpl{
|
||||
watcherBGNotifier: syncutil.NewAsyncTaskNotifier[struct{}](),
|
||||
pendingEvents: make(map[string]*message.BroadcastEvent, 0),
|
||||
broadcasterImpl: broadcaster,
|
||||
version: 0,
|
||||
input: make(chan *message.BroadcastEvent),
|
||||
output: make(chan *message.BroadcastEvent),
|
||||
}
|
||||
go w.execute()
|
||||
return w
|
||||
}
|
||||
|
||||
// watcherImpl implement the Watcher interface.
|
||||
type watcherImpl struct {
|
||||
watcherBGNotifier *syncutil.AsyncTaskNotifier[struct{}]
|
||||
pendingEvents map[string]*message.BroadcastEvent
|
||||
*broadcasterImpl
|
||||
version int
|
||||
input chan *message.BroadcastEvent
|
||||
output chan *message.BroadcastEvent
|
||||
}
|
||||
|
||||
func (w *watcherImpl) ObserveResourceKeyEvent(ctx context.Context, ev *message.BroadcastEvent) error {
|
||||
select {
|
||||
case w.input <- ev:
|
||||
return nil
|
||||
case <-w.backgroundTaskNotifier.Context().Done():
|
||||
return w.backgroundTaskNotifier.Context().Err()
|
||||
case <-w.watcherBGNotifier.Context().Done():
|
||||
return w.watcherBGNotifier.Context().Err()
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
}
|
||||
}
|
||||
|
||||
func (w *watcherImpl) EventChan() <-chan *message.BroadcastEvent {
|
||||
return w.output
|
||||
}
|
||||
|
||||
func (w *watcherImpl) execute() {
|
||||
defer func() {
|
||||
close(w.output)
|
||||
w.watcherBGNotifier.Finish(struct{}{})
|
||||
}()
|
||||
for {
|
||||
ch := w.manager.WatchAtVersion(w.version)
|
||||
select {
|
||||
case <-w.backgroundTaskNotifier.Context().Done():
|
||||
w.Logger().Info("watcher is exit because of broadcaseter is closing", zap.Int("version", w.version))
|
||||
return
|
||||
case <-w.watcherBGNotifier.Context().Done():
|
||||
w.Logger().Info("watcher is exit because of watcher itself is closing", zap.Int("version", w.version))
|
||||
return
|
||||
case <-ch:
|
||||
w.update()
|
||||
case ev := <-w.input:
|
||||
w.pendingEvents[message.UniqueKeyOfBroadcastEvent(ev)] = ev
|
||||
w.update()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (w *watcherImpl) update() {
|
||||
w.version = w.manager.CurrentVersion()
|
||||
newPendingEvent := make(map[string]*message.BroadcastEvent, len(w.pendingEvents))
|
||||
for key, pendingEvent := range w.pendingEvents {
|
||||
switch ev := pendingEvent.Event.(type) {
|
||||
case *messagespb.BroadcastEvent_ResourceKeyAckAll:
|
||||
task, ok := w.manager.GetBroadcastTaskByResourceKey(message.NewResourceKeyFromProto(ev.ResourceKeyAckAll.ResourceKey))
|
||||
if !ok || task.IsAllAcked() {
|
||||
w.output <- pendingEvent
|
||||
continue
|
||||
}
|
||||
case *messagespb.BroadcastEvent_ResourceKeyAckOne:
|
||||
task, ok := w.manager.GetBroadcastTaskByResourceKey(message.NewResourceKeyFromProto(ev.ResourceKeyAckOne.ResourceKey))
|
||||
if !ok || task.IsAcked() {
|
||||
w.output <- pendingEvent
|
||||
continue
|
||||
}
|
||||
}
|
||||
newPendingEvent[key] = pendingEvent
|
||||
}
|
||||
w.pendingEvents = newPendingEvent
|
||||
}
|
||||
|
||||
func (w *watcherImpl) Close() {
|
||||
w.watcherBGNotifier.Cancel()
|
||||
w.watcherBGNotifier.BlockUntilFinish()
|
||||
}
|
|
@ -4,7 +4,6 @@ import (
|
|||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/service/broadcast"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||
|
@ -62,18 +61,3 @@ func (s *broadcastServceImpl) Ack(ctx context.Context, req *streamingpb.Broadcas
|
|||
}
|
||||
return &streamingpb.BroadcastAckResponse{}, nil
|
||||
}
|
||||
|
||||
func (s *broadcastServceImpl) Watch(svr streamingpb.StreamingCoordBroadcastService_WatchServer) error {
|
||||
broadcaster, err := s.broadcaster.GetWithContext(svr.Context())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
watcher, err := broadcaster.NewWatcher()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer watcher.Close()
|
||||
|
||||
server := broadcast.NewBroadcastWatchServer(watcher, svr)
|
||||
return server.Execute()
|
||||
}
|
||||
|
|
|
@ -1,30 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
type broadcastWatchGrpcServerHelper struct {
|
||||
streamingpb.StreamingCoordBroadcastService_WatchServer
|
||||
}
|
||||
|
||||
// SendResourceKeyEvent sends the resource key event to client.
|
||||
func (h *broadcastWatchGrpcServerHelper) SendResourceKeyEvent(ev *message.BroadcastEvent) error {
|
||||
return h.Send(&streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_EventDone{
|
||||
EventDone: &streamingpb.BroadcastEventWatchResponse{
|
||||
Event: ev,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
// SendCloseResponse sends the close response to client.
|
||||
func (h *broadcastWatchGrpcServerHelper) SendCloseResponse() error {
|
||||
return h.Send(&streamingpb.BroadcastWatchResponse{
|
||||
Response: &streamingpb.BroadcastWatchResponse_Close{
|
||||
Close: &streamingpb.CloseBroadcastWatchResponse{},
|
||||
},
|
||||
})
|
||||
}
|
|
@ -1,117 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
var errClosedByUser = errors.New("closed by user")
|
||||
|
||||
func NewBroadcastWatchServer(
|
||||
w broadcaster.Watcher,
|
||||
streamServer streamingpb.StreamingCoordBroadcastService_WatchServer,
|
||||
) *BroadcastWatchServer {
|
||||
ctx, cancel := context.WithCancelCause(streamServer.Context())
|
||||
s := &BroadcastWatchServer{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
w: w,
|
||||
streamServer: broadcastWatchGrpcServerHelper{
|
||||
streamServer,
|
||||
},
|
||||
}
|
||||
s.SetLogger(resource.Resource().Logger().With(log.FieldComponent("broadcast-watch-server")))
|
||||
return s
|
||||
}
|
||||
|
||||
type BroadcastWatchServer struct {
|
||||
log.Binder
|
||||
ctx context.Context
|
||||
cancel context.CancelCauseFunc
|
||||
w broadcaster.Watcher
|
||||
streamServer broadcastWatchGrpcServerHelper
|
||||
}
|
||||
|
||||
func (s *BroadcastWatchServer) Execute() error {
|
||||
// Start a recv arm to handle the control message from client.
|
||||
go func() {
|
||||
// recv loop will be blocked until the stream is closed.
|
||||
// 1. close by client.
|
||||
// 2. close by server context cancel by return of outside Execute.
|
||||
_ = s.recvLoop()
|
||||
}()
|
||||
|
||||
// Start a send loop on current main goroutine.
|
||||
// the loop will be blocked until:
|
||||
// 1. the stream is broken.
|
||||
// 2. recv arm recv closed and all response is sent.
|
||||
return s.sendLoop()
|
||||
}
|
||||
|
||||
// recvLoop receives the message from client.
|
||||
func (s *BroadcastWatchServer) recvLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
s.cancel(err)
|
||||
s.Logger().Warn("recv arm of stream closed by unexpected error", zap.Error(err))
|
||||
return
|
||||
}
|
||||
s.cancel(errClosedByUser)
|
||||
s.Logger().Info("recv arm of stream closed")
|
||||
}()
|
||||
|
||||
for {
|
||||
req, err := s.streamServer.Recv()
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
switch req := req.Command.(type) {
|
||||
case *streamingpb.BroadcastWatchRequest_CreateEventWatch:
|
||||
// Add new incoming resource key int watcher.
|
||||
s.w.ObserveResourceKeyEvent(s.streamServer.Context(), req.CreateEventWatch.Event)
|
||||
case *streamingpb.BroadcastWatchRequest_Close:
|
||||
// Ignore the command, the stream will be closed by client with io.EOF
|
||||
default:
|
||||
s.Logger().Warn("unknown command type ignored", zap.Any("command", req))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// sendLoop sends the message to client.
|
||||
func (s *BroadcastWatchServer) sendLoop() (err error) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
s.Logger().Warn("send arm of stream closed by unexpected error", zap.Error(err))
|
||||
return
|
||||
}
|
||||
s.Logger().Info("send arm of stream closed")
|
||||
}()
|
||||
for {
|
||||
select {
|
||||
case ev, ok := <-s.w.EventChan():
|
||||
if !ok {
|
||||
return errors.New("watcher is closed")
|
||||
}
|
||||
if err := s.streamServer.SendResourceKeyEvent(ev); err != nil {
|
||||
return err
|
||||
}
|
||||
case <-s.ctx.Done():
|
||||
err := context.Cause(s.ctx)
|
||||
if errors.Is(err, errClosedByUser) {
|
||||
return s.streamServer.SendCloseResponse()
|
||||
}
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,76 +0,0 @@
|
|||
package broadcast
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_broadcaster"
|
||||
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
|
||||
"github.com/milvus-io/milvus/pkg/v2/mocks/proto/mock_streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/messagespb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
)
|
||||
|
||||
func TestBroadcastWatch(t *testing.T) {
|
||||
resource.InitForTest()
|
||||
w := mock_broadcaster.NewMockWatcher(t)
|
||||
input := make(chan *message.BroadcastEvent, 5)
|
||||
output := make(chan *message.BroadcastEvent, 5)
|
||||
w.EXPECT().ObserveResourceKeyEvent(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, ev *messagespb.BroadcastEvent) error {
|
||||
output <- ev
|
||||
return nil
|
||||
})
|
||||
w.EXPECT().EventChan().Return(output)
|
||||
streamServer := mock_streamingpb.NewMockStreamingCoordBroadcastService_WatchServer(t)
|
||||
streamServer.EXPECT().Context().Return(context.Background())
|
||||
closed := false
|
||||
streamServer.EXPECT().Recv().RunAndReturn(func() (*streamingpb.BroadcastWatchRequest, error) {
|
||||
if closed {
|
||||
return nil, io.EOF
|
||||
}
|
||||
ev, ok := <-input
|
||||
if !ok {
|
||||
closed = true
|
||||
return &streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_Close{
|
||||
Close: &streamingpb.CloseBroadcastWatchRequest{},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
return &streamingpb.BroadcastWatchRequest{
|
||||
Command: &streamingpb.BroadcastWatchRequest_CreateEventWatch{
|
||||
CreateEventWatch: &streamingpb.BroadcastCreateEventWatchRequest{
|
||||
Event: ev,
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
})
|
||||
|
||||
streamOutput := make(chan *message.BroadcastEvent, 5)
|
||||
streamServer.EXPECT().Send(mock.Anything).RunAndReturn(func(bwr *streamingpb.BroadcastWatchResponse) error {
|
||||
if bwr.GetEventDone() != nil {
|
||||
streamOutput <- bwr.GetEventDone().Event
|
||||
}
|
||||
return nil
|
||||
})
|
||||
s := NewBroadcastWatchServer(w, streamServer)
|
||||
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c1"))
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c2"))
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c3"))
|
||||
input <- message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("c4"))
|
||||
done := make(chan struct{})
|
||||
go func() {
|
||||
s.Execute()
|
||||
close(done)
|
||||
}()
|
||||
for i := 0; i < 4; i++ {
|
||||
<-streamOutput
|
||||
}
|
||||
close(input)
|
||||
<-done
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -173,79 +173,6 @@ func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) RunAndReturn(
|
|||
return _c
|
||||
}
|
||||
|
||||
// Watch provides a mock function with given fields: ctx, opts
|
||||
func (_m *MockStreamingCoordBroadcastServiceClient) Watch(ctx context.Context, opts ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Watch")
|
||||
}
|
||||
|
||||
var r0 streamingpb.StreamingCoordBroadcastService_WatchClient
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error)); ok {
|
||||
return rf(ctx, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, ...grpc.CallOption) streamingpb.StreamingCoordBroadcastService_WatchClient); ok {
|
||||
r0 = rf(ctx, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(streamingpb.StreamingCoordBroadcastService_WatchClient)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastServiceClient_Watch_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Watch'
|
||||
type MockStreamingCoordBroadcastServiceClient_Watch_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Watch is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Watch(ctx interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
return &MockStreamingCoordBroadcastServiceClient_Watch_Call{Call: _e.mock.On("Watch",
|
||||
append([]interface{}{ctx}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) Run(run func(ctx context.Context, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-1)
|
||||
for i, a := range args[1:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) Return(_a0 streamingpb.StreamingCoordBroadcastService_WatchClient, _a1 error) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastServiceClient_Watch_Call) RunAndReturn(run func(context.Context, ...grpc.CallOption) (streamingpb.StreamingCoordBroadcastService_WatchClient, error)) *MockStreamingCoordBroadcastServiceClient_Watch_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockStreamingCoordBroadcastServiceClient creates a new instance of MockStreamingCoordBroadcastServiceClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockStreamingCoordBroadcastServiceClient(t interface {
|
||||
|
|
|
@ -1,430 +0,0 @@
|
|||
// Code generated by mockery v2.46.0. DO NOT EDIT.
|
||||
|
||||
package mock_streamingpb
|
||||
|
||||
import (
|
||||
context "context"
|
||||
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
metadata "google.golang.org/grpc/metadata"
|
||||
|
||||
streamingpb "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient is an autogenerated mock type for the StreamingCoordBroadcastService_WatchClient type
|
||||
type MockStreamingCoordBroadcastService_WatchClient struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Expecter struct {
|
||||
mock *mock.Mock
|
||||
}
|
||||
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) EXPECT() *MockStreamingCoordBroadcastService_WatchClient_Expecter {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// CloseSend provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) CloseSend() error {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for CloseSend")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func() error); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CloseSend'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// CloseSend is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) CloseSend() *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call{Call: _e.mock.On("CloseSend")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call) RunAndReturn(run func() error) *MockStreamingCoordBroadcastService_WatchClient_CloseSend_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Context provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Context() context.Context {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Context")
|
||||
}
|
||||
|
||||
var r0 context.Context
|
||||
if rf, ok := ret.Get(0).(func() context.Context); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(context.Context)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Context_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Context is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Context() *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Context_Call{Call: _e.mock.On("Context")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) Return(_a0 context.Context) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Context_Call) RunAndReturn(run func() context.Context) *MockStreamingCoordBroadcastService_WatchClient_Context_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Header provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Header() (metadata.MD, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Header")
|
||||
}
|
||||
|
||||
var r0 metadata.MD
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (metadata.MD, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() metadata.MD); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(metadata.MD)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Header_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Header'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Header_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Header is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Header() *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Header_Call{Call: _e.mock.On("Header")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) Return(_a0 metadata.MD, _a1 error) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Header_Call) RunAndReturn(run func() (metadata.MD, error)) *MockStreamingCoordBroadcastService_WatchClient_Header_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Recv provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Recv() (*streamingpb.BroadcastWatchResponse, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Recv")
|
||||
}
|
||||
|
||||
var r0 *streamingpb.BroadcastWatchResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (*streamingpb.BroadcastWatchResponse, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() *streamingpb.BroadcastWatchResponse); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*streamingpb.BroadcastWatchResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Recv_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Recv is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Recv() *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Recv_Call{Call: _e.mock.On("Recv")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) Return(_a0 *streamingpb.BroadcastWatchResponse, _a1 error) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Recv_Call) RunAndReturn(run func() (*streamingpb.BroadcastWatchResponse, error)) *MockStreamingCoordBroadcastService_WatchClient_Recv_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// RecvMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) RecvMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for RecvMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// RecvMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) RecvMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchClient_RecvMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Send provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Send(_a0 *streamingpb.BroadcastWatchRequest) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Send")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(*streamingpb.BroadcastWatchRequest) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Send_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Send'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Send_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Send is a helper method to define mock.On call
|
||||
// - _a0 *streamingpb.BroadcastWatchRequest
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Send(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Send_Call{Call: _e.mock.On("Send", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) Run(run func(_a0 *streamingpb.BroadcastWatchRequest)) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*streamingpb.BroadcastWatchRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Send_Call) RunAndReturn(run func(*streamingpb.BroadcastWatchRequest) error) *MockStreamingCoordBroadcastService_WatchClient_Send_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SendMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) SendMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SendMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SendMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) SendMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call{Call: _e.mock.On("SendMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchClient_SendMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Trailer provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchClient) Trailer() metadata.MD {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Trailer")
|
||||
}
|
||||
|
||||
var r0 metadata.MD
|
||||
if rf, ok := ret.Get(0).(func() metadata.MD); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(metadata.MD)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchClient_Trailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Trailer'
|
||||
type MockStreamingCoordBroadcastService_WatchClient_Trailer_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Trailer is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchClient_Expecter) Trailer() *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchClient_Trailer_Call{Call: _e.mock.On("Trailer")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) Return(_a0 metadata.MD) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call) RunAndReturn(run func() metadata.MD) *MockStreamingCoordBroadcastService_WatchClient_Trailer_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockStreamingCoordBroadcastService_WatchClient creates a new instance of MockStreamingCoordBroadcastService_WatchClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockStreamingCoordBroadcastService_WatchClient(t interface {
|
||||
mock.TestingT
|
||||
Cleanup(func())
|
||||
}) *MockStreamingCoordBroadcastService_WatchClient {
|
||||
mock := &MockStreamingCoordBroadcastService_WatchClient{}
|
||||
mock.Mock.Test(t)
|
||||
|
||||
t.Cleanup(func() { mock.AssertExpectations(t) })
|
||||
|
||||
return mock
|
||||
}
|
|
@ -1,406 +0,0 @@
|
|||
// Code generated by mockery v2.46.0. DO NOT EDIT.
|
||||
|
||||
package mock_streamingpb
|
||||
|
||||
import (
|
||||
context "context"
|
||||
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
metadata "google.golang.org/grpc/metadata"
|
||||
|
||||
streamingpb "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer is an autogenerated mock type for the StreamingCoordBroadcastService_WatchServer type
|
||||
type MockStreamingCoordBroadcastService_WatchServer struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Expecter struct {
|
||||
mock *mock.Mock
|
||||
}
|
||||
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) EXPECT() *MockStreamingCoordBroadcastService_WatchServer_Expecter {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// Context provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) Context() context.Context {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Context")
|
||||
}
|
||||
|
||||
var r0 context.Context
|
||||
if rf, ok := ret.Get(0).(func() context.Context); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(context.Context)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_Context_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Context'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Context_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Context is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Context() *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Context_Call{Call: _e.mock.On("Context")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) Return(_a0 context.Context) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Context_Call) RunAndReturn(run func() context.Context) *MockStreamingCoordBroadcastService_WatchServer_Context_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Recv provides a mock function with given fields:
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) Recv() (*streamingpb.BroadcastWatchRequest, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Recv")
|
||||
}
|
||||
|
||||
var r0 *streamingpb.BroadcastWatchRequest
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (*streamingpb.BroadcastWatchRequest, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() *streamingpb.BroadcastWatchRequest); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*streamingpb.BroadcastWatchRequest)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_Recv_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Recv'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Recv_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Recv is a helper method to define mock.On call
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Recv() *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Recv_Call{Call: _e.mock.On("Recv")}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) Run(run func()) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) Return(_a0 *streamingpb.BroadcastWatchRequest, _a1 error) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Recv_Call) RunAndReturn(run func() (*streamingpb.BroadcastWatchRequest, error)) *MockStreamingCoordBroadcastService_WatchServer_Recv_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// RecvMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) RecvMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for RecvMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RecvMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// RecvMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) RecvMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call{Call: _e.mock.On("RecvMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchServer_RecvMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Send provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) Send(_a0 *streamingpb.BroadcastWatchResponse) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Send")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(*streamingpb.BroadcastWatchResponse) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_Send_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Send'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_Send_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Send is a helper method to define mock.On call
|
||||
// - _a0 *streamingpb.BroadcastWatchResponse
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) Send(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_Send_Call{Call: _e.mock.On("Send", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) Run(run func(_a0 *streamingpb.BroadcastWatchResponse)) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*streamingpb.BroadcastWatchResponse))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_Send_Call) RunAndReturn(run func(*streamingpb.BroadcastWatchResponse) error) *MockStreamingCoordBroadcastService_WatchServer_Send_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SendHeader provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SendHeader(_a0 metadata.MD) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SendHeader")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(metadata.MD) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendHeader'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SendHeader is a helper method to define mock.On call
|
||||
// - _a0 metadata.MD
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SendHeader(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call{Call: _e.mock.On("SendHeader", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(metadata.MD))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call) RunAndReturn(run func(metadata.MD) error) *MockStreamingCoordBroadcastService_WatchServer_SendHeader_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SendMsg provides a mock function with given fields: m
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SendMsg(m interface{}) error {
|
||||
ret := _m.Called(m)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SendMsg")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(interface{}) error); ok {
|
||||
r0 = rf(m)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SendMsg'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SendMsg is a helper method to define mock.On call
|
||||
// - m interface{}
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SendMsg(m interface{}) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call{Call: _e.mock.On("SendMsg", m)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) Run(run func(m interface{})) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(interface{}))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call) RunAndReturn(run func(interface{}) error) *MockStreamingCoordBroadcastService_WatchServer_SendMsg_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetHeader provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SetHeader(_a0 metadata.MD) error {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for SetHeader")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(metadata.MD) error); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetHeader'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SetHeader is a helper method to define mock.On call
|
||||
// - _a0 metadata.MD
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SetHeader(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call{Call: _e.mock.On("SetHeader", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(metadata.MD))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) Return(_a0 error) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call) RunAndReturn(run func(metadata.MD) error) *MockStreamingCoordBroadcastService_WatchServer_SetHeader_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetTrailer provides a mock function with given fields: _a0
|
||||
func (_m *MockStreamingCoordBroadcastService_WatchServer) SetTrailer(_a0 metadata.MD) {
|
||||
_m.Called(_a0)
|
||||
}
|
||||
|
||||
// MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetTrailer'
|
||||
type MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SetTrailer is a helper method to define mock.On call
|
||||
// - _a0 metadata.MD
|
||||
func (_e *MockStreamingCoordBroadcastService_WatchServer_Expecter) SetTrailer(_a0 interface{}) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
return &MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call{Call: _e.mock.On("SetTrailer", _a0)}
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) Run(run func(_a0 metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(metadata.MD))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) Return() *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call) RunAndReturn(run func(metadata.MD)) *MockStreamingCoordBroadcastService_WatchServer_SetTrailer_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockStreamingCoordBroadcastService_WatchServer creates a new instance of MockStreamingCoordBroadcastService_WatchServer. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockStreamingCoordBroadcastService_WatchServer(t interface {
|
||||
mock.TestingT
|
||||
Cleanup(func())
|
||||
}) *MockStreamingCoordBroadcastService_WatchServer {
|
||||
mock := &MockStreamingCoordBroadcastService_WatchServer{}
|
||||
mock.Mock.Test(t)
|
||||
|
||||
t.Cleanup(func() { mock.AssertExpectations(t) })
|
||||
|
||||
return mock
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
|
|
|
@ -96,9 +96,6 @@ service StreamingCoordBroadcastService {
|
|||
|
||||
// Ack acknowledge broadcast message is consumed.
|
||||
rpc Ack(BroadcastAckRequest) returns (BroadcastAckResponse) {}
|
||||
|
||||
// Watch watch the broadcast events.
|
||||
rpc Watch(stream BroadcastWatchRequest) returns (stream BroadcastWatchResponse) {}
|
||||
}
|
||||
|
||||
// BroadcastRequest is the request of the Broadcast RPC.
|
||||
|
@ -120,34 +117,6 @@ message BroadcastAckRequest {
|
|||
message BroadcastAckResponse {
|
||||
}
|
||||
|
||||
message BroadcastWatchRequest {
|
||||
oneof command {
|
||||
BroadcastCreateEventWatchRequest create_event_watch = 1;
|
||||
CloseBroadcastWatchRequest close = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message BroadcastCreateEventWatchRequest {
|
||||
messages.BroadcastEvent event = 1;
|
||||
}
|
||||
|
||||
message CloseBroadcastWatchRequest {
|
||||
}
|
||||
|
||||
message BroadcastWatchResponse {
|
||||
oneof response {
|
||||
BroadcastEventWatchResponse event_done = 1;
|
||||
CloseBroadcastWatchResponse close = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message BroadcastEventWatchResponse {
|
||||
messages.BroadcastEvent event = 1;
|
||||
}
|
||||
|
||||
message CloseBroadcastWatchResponse {
|
||||
}
|
||||
|
||||
//
|
||||
// StreamingCoordAssignmentService
|
||||
//
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -110,7 +110,6 @@ var StreamingNodeStateService_ServiceDesc = grpc.ServiceDesc{
|
|||
const (
|
||||
StreamingCoordBroadcastService_Broadcast_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Broadcast"
|
||||
StreamingCoordBroadcastService_Ack_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Ack"
|
||||
StreamingCoordBroadcastService_Watch_FullMethodName = "/milvus.proto.streaming.StreamingCoordBroadcastService/Watch"
|
||||
)
|
||||
|
||||
// StreamingCoordBroadcastServiceClient is the client API for StreamingCoordBroadcastService service.
|
||||
|
@ -122,8 +121,6 @@ type StreamingCoordBroadcastServiceClient interface {
|
|||
Broadcast(ctx context.Context, in *BroadcastRequest, opts ...grpc.CallOption) (*BroadcastResponse, error)
|
||||
// Ack acknowledge broadcast message is consumed.
|
||||
Ack(ctx context.Context, in *BroadcastAckRequest, opts ...grpc.CallOption) (*BroadcastAckResponse, error)
|
||||
// Watch watch the broadcast events.
|
||||
Watch(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordBroadcastService_WatchClient, error)
|
||||
}
|
||||
|
||||
type streamingCoordBroadcastServiceClient struct {
|
||||
|
@ -152,37 +149,6 @@ func (c *streamingCoordBroadcastServiceClient) Ack(ctx context.Context, in *Broa
|
|||
return out, nil
|
||||
}
|
||||
|
||||
func (c *streamingCoordBroadcastServiceClient) Watch(ctx context.Context, opts ...grpc.CallOption) (StreamingCoordBroadcastService_WatchClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &StreamingCoordBroadcastService_ServiceDesc.Streams[0], StreamingCoordBroadcastService_Watch_FullMethodName, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &streamingCoordBroadcastServiceWatchClient{stream}
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type StreamingCoordBroadcastService_WatchClient interface {
|
||||
Send(*BroadcastWatchRequest) error
|
||||
Recv() (*BroadcastWatchResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type streamingCoordBroadcastServiceWatchClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchClient) Send(m *BroadcastWatchRequest) error {
|
||||
return x.ClientStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchClient) Recv() (*BroadcastWatchResponse, error) {
|
||||
m := new(BroadcastWatchResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// StreamingCoordBroadcastServiceServer is the server API for StreamingCoordBroadcastService service.
|
||||
// All implementations should embed UnimplementedStreamingCoordBroadcastServiceServer
|
||||
// for forward compatibility
|
||||
|
@ -192,8 +158,6 @@ type StreamingCoordBroadcastServiceServer interface {
|
|||
Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error)
|
||||
// Ack acknowledge broadcast message is consumed.
|
||||
Ack(context.Context, *BroadcastAckRequest) (*BroadcastAckResponse, error)
|
||||
// Watch watch the broadcast events.
|
||||
Watch(StreamingCoordBroadcastService_WatchServer) error
|
||||
}
|
||||
|
||||
// UnimplementedStreamingCoordBroadcastServiceServer should be embedded to have forward compatible implementations.
|
||||
|
@ -206,9 +170,6 @@ func (UnimplementedStreamingCoordBroadcastServiceServer) Broadcast(context.Conte
|
|||
func (UnimplementedStreamingCoordBroadcastServiceServer) Ack(context.Context, *BroadcastAckRequest) (*BroadcastAckResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Ack not implemented")
|
||||
}
|
||||
func (UnimplementedStreamingCoordBroadcastServiceServer) Watch(StreamingCoordBroadcastService_WatchServer) error {
|
||||
return status.Errorf(codes.Unimplemented, "method Watch not implemented")
|
||||
}
|
||||
|
||||
// UnsafeStreamingCoordBroadcastServiceServer may be embedded to opt out of forward compatibility for this service.
|
||||
// Use of this interface is not recommended, as added methods to StreamingCoordBroadcastServiceServer will
|
||||
|
@ -257,32 +218,6 @@ func _StreamingCoordBroadcastService_Ack_Handler(srv interface{}, ctx context.Co
|
|||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _StreamingCoordBroadcastService_Watch_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
return srv.(StreamingCoordBroadcastServiceServer).Watch(&streamingCoordBroadcastServiceWatchServer{stream})
|
||||
}
|
||||
|
||||
type StreamingCoordBroadcastService_WatchServer interface {
|
||||
Send(*BroadcastWatchResponse) error
|
||||
Recv() (*BroadcastWatchRequest, error)
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type streamingCoordBroadcastServiceWatchServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchServer) Send(m *BroadcastWatchResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *streamingCoordBroadcastServiceWatchServer) Recv() (*BroadcastWatchRequest, error) {
|
||||
m := new(BroadcastWatchRequest)
|
||||
if err := x.ServerStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// StreamingCoordBroadcastService_ServiceDesc is the grpc.ServiceDesc for StreamingCoordBroadcastService service.
|
||||
// It's only intended for direct use with grpc.RegisterService,
|
||||
// and not to be introspected or modified (even as a copy)
|
||||
|
@ -299,14 +234,7 @@ var StreamingCoordBroadcastService_ServiceDesc = grpc.ServiceDesc{
|
|||
Handler: _StreamingCoordBroadcastService_Ack_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{
|
||||
{
|
||||
StreamName: "Watch",
|
||||
Handler: _StreamingCoordBroadcastService_Watch_Handler,
|
||||
ServerStreams: true,
|
||||
ClientStreams: true,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Metadata: "streaming.proto",
|
||||
}
|
||||
|
||||
|
|
|
@ -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(),
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue