diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index ff51bd9967..51daaf4ebe 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -1445,7 +1445,6 @@ func TestImportV2(t *testing.T) { wal := mock_streaming.NewMockWALAccesser(t) b := mock_streaming.NewMockBroadcast(t) wal.EXPECT().Broadcast().Return(b).Maybe() - b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() streaming.SetWALForTest(wal) defer streaming.RecoverWALForTest() diff --git a/internal/distributed/streaming/broadcast.go b/internal/distributed/streaming/broadcast.go index c7afb18941..2de44b4193 100644 --- a/internal/distributed/streaming/broadcast.go +++ b/internal/distributed/streaming/broadcast.go @@ -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)) -} diff --git a/internal/distributed/streaming/streaming.go b/internal/distributed/streaming/streaming.go index 9da127ad06..a22f563c06 100644 --- a/internal/distributed/streaming/streaming.go +++ b/internal/distributed/streaming/streaming.go @@ -122,20 +122,14 @@ type WALAccesser interface { type Broadcast interface { // Append of Broadcast sends a broadcast message to all target vchannels. // Guarantees the atomicity written of the messages and eventual consistency. - // The resource-key bound at the message will be held until the message is acked at consumer. - // Once the resource-key is held, the append operation will be rejected. - // Use resource-key to make a sequential operation at same resource-key. + // The resource-key bound at the message will be held as a mutex until the message is broadcasted to all vchannels, + // so the other append operation with the same resource-key will be searialized with a deterministic order on every vchannel. + // The Append operation will be blocked until the message is consumed and acknowledged by the flusher at streamingnode. Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) // Ack acknowledges a broadcast message at the specified vchannel. // It must be called after the message is comsumed by the unique-consumer. Ack(ctx context.Context, req types.BroadcastAckRequest) error - - // BlockUntilResourceKeyAckOnce blocks until the resource-key-bind broadcast message is acked at any one vchannel. - BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error - - // BlockUntilResourceKeyAckOnce blocks until the resource-key-bind broadcast message is acked at all vchannel. - BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error } // Txn is the interface for writing transaction into the wal. diff --git a/internal/distributed/streaming/streaming_test.go b/internal/distributed/streaming/streaming_test.go index 5ad171e9b7..52760628af 100644 --- a/internal/distributed/streaming/streaming_test.go +++ b/internal/distributed/streaming/streaming_test.go @@ -16,7 +16,6 @@ import ( "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/v2/streaming/util/options" - "github.com/milvus-io/milvus/pkg/v2/streaming/util/types" _ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/pulsar" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" ) @@ -40,9 +39,6 @@ func TestStreamingBroadcast(t *testing.T) { streaming.Init() defer streaming.Release() - err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(context.Background(), message.NewCollectionNameResourceKey(collectionName)) - assert.NoError(t, err) - msg, _ := message.NewCreateCollectionMessageBuilderV1(). WithHeader(&message.CreateCollectionMessageHeader{ CollectionId: 1, @@ -69,53 +65,6 @@ func TestStreamingBroadcast(t *testing.T) { assert.Error(t, err) assert.True(t, status.AsStreamingError(err).IsResourceAcquired()) assert.Nil(t, resp2) - - // resource key should be block until ack. - ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName)) - assert.ErrorIs(t, err, context.DeadlineExceeded) - - ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName)) - assert.ErrorIs(t, err, context.DeadlineExceeded) - - err = streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{ - BroadcastID: resp.BroadcastID, - VChannel: vChannels[0], - }) - assert.NoError(t, err) - - // all should be blocked - ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName)) - assert.ErrorIs(t, err, context.DeadlineExceeded) - - // once should be returned - ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName)) - assert.NoError(t, err) - - err = streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{ - BroadcastID: resp.BroadcastID, - VChannel: vChannels[1], - }) - assert.NoError(t, err) - - // all should be blocked - ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey(collectionName)) - assert.NoError(t, err) - - // once should be returned - ctx, cancel = context.WithTimeout(context.Background(), time.Millisecond*100) - defer cancel() - err = streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey(collectionName)) - assert.NoError(t, err) } func TestStreamingProduce(t *testing.T) { diff --git a/internal/distributed/streaming/wal_test.go b/internal/distributed/streaming/wal_test.go index 7f3a4d1f3d..d80377fdd5 100644 --- a/internal/distributed/streaming/wal_test.go +++ b/internal/distributed/streaming/wal_test.go @@ -46,7 +46,6 @@ func TestWAL(t *testing.T) { }, nil }) broadcastServce.EXPECT().Ack(mock.Anything, mock.Anything).Return(nil) - broadcastServce.EXPECT().BlockUntilEvent(mock.Anything, mock.Anything).Return(nil) coordClient.EXPECT().Broadcast().Return(broadcastServce) handler := mock_handler.NewMockHandlerClient(t) handler.EXPECT().Close().Return() @@ -139,12 +138,6 @@ func TestWAL(t *testing.T) { err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1}) assert.NoError(t, err) - err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1")) - assert.NoError(t, err) - - err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2")) - assert.NoError(t, err) - w.Close() w.GetLatestMVCCTimestampIfLocal(ctx, vChannel1) @@ -158,12 +151,6 @@ func TestWAL(t *testing.T) { err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1}) assert.Error(t, err) - - err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1")) - assert.Error(t, err) - - err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2")) - assert.Error(t, err) } func newInsertMessage(vChannel string) message.MutableMessage { diff --git a/internal/mocks/distributed/mock_streaming/mock_Broadcast.go b/internal/mocks/distributed/mock_streaming/mock_Broadcast.go index 37e19e554e..1e78402966 100644 --- a/internal/mocks/distributed/mock_streaming/mock_Broadcast.go +++ b/internal/mocks/distributed/mock_streaming/mock_Broadcast.go @@ -130,100 +130,6 @@ func (_c *MockBroadcast_Append_Call) RunAndReturn(run func(context.Context, mess return _c } -// BlockUntilResourceKeyAckAll provides a mock function with given fields: ctx, rk -func (_m *MockBroadcast) BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error { - ret := _m.Called(ctx, rk) - - if len(ret) == 0 { - panic("no return value specified for BlockUntilResourceKeyAckAll") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok { - r0 = rf(ctx, rk) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockBroadcast_BlockUntilResourceKeyAckAll_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckAll' -type MockBroadcast_BlockUntilResourceKeyAckAll_Call struct { - *mock.Call -} - -// BlockUntilResourceKeyAckAll is a helper method to define mock.On call -// - ctx context.Context -// - rk message.ResourceKey -func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckAll(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { - return &MockBroadcast_BlockUntilResourceKeyAckAll_Call{Call: _e.mock.On("BlockUntilResourceKeyAckAll", ctx, rk)} -} - -func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(message.ResourceKey)) - }) - return _c -} - -func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { - _c.Call.Return(run) - return _c -} - -// BlockUntilResourceKeyAckOnce provides a mock function with given fields: ctx, rk -func (_m *MockBroadcast) BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error { - ret := _m.Called(ctx, rk) - - if len(ret) == 0 { - panic("no return value specified for BlockUntilResourceKeyAckOnce") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok { - r0 = rf(ctx, rk) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockBroadcast_BlockUntilResourceKeyAckOnce_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckOnce' -type MockBroadcast_BlockUntilResourceKeyAckOnce_Call struct { - *mock.Call -} - -// BlockUntilResourceKeyAckOnce is a helper method to define mock.On call -// - ctx context.Context -// - rk message.ResourceKey -func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckOnce(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { - return &MockBroadcast_BlockUntilResourceKeyAckOnce_Call{Call: _e.mock.On("BlockUntilResourceKeyAckOnce", ctx, rk)} -} - -func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(message.ResourceKey)) - }) - return _c -} - -func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { - _c.Call.Return(run) - return _c -} - // NewMockBroadcast creates a new instance of MockBroadcast. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewMockBroadcast(t interface { diff --git a/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go b/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go index 4570458b8f..9544c099cd 100644 --- a/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go +++ b/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go @@ -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) diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index f3130a4ae3..dd0239705b 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -41,7 +41,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/proxy/connection" "github.com/milvus-io/milvus/internal/types" @@ -56,7 +55,6 @@ import ( "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/proxypb" "github.com/milvus-io/milvus/pkg/v2/proto/querypb" - "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" "github.com/milvus-io/milvus/pkg/v2/util" "github.com/milvus-io/milvus/pkg/v2/util/commonpbutil" "github.com/milvus-io/milvus/pkg/v2/util/crypto" @@ -6573,21 +6571,6 @@ func (node *Proxy) ImportV2(ctx context.Context, req *internalpb.ImportRequest) return resp, nil } - // Import is asynchronous consumed from the wal, so we need to wait for the wal to release the resource key. - // The job can be seen by the user after the resource key is acked once at any vchannel. - jobID, err := strconv.ParseInt(resp.GetJobID(), 10, 64) - if err != nil { - return &internalpb.ImportResponse{ - Status: merr.Status(merr.WrapErrServiceInternal("invalid job ID")), - }, nil - } - resourceKey := message.NewImportJobIDResourceKey(jobID) - if err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, resourceKey); err != nil { - log.Warn("failed to wait for resource key ack", zap.Error(err)) - return &internalpb.ImportResponse{ - Status: merr.Status(merr.WrapErrServiceInternal("failed to wait for resource key ack")), - }, nil - } metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.SuccessLabel, req.GetDbName(), req.GetCollectionName()).Inc() metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds())) return resp, nil diff --git a/internal/proxy/impl_test.go b/internal/proxy/impl_test.go index 62627cb60a..d34d565b83 100644 --- a/internal/proxy/impl_test.go +++ b/internal/proxy/impl_test.go @@ -1617,7 +1617,6 @@ func TestProxy_ImportV2(t *testing.T) { wal := mock_streaming.NewMockWALAccesser(t) b := mock_streaming.NewMockBroadcast(t) wal.EXPECT().Broadcast().Return(b).Maybe() - b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Maybe() streaming.SetWALForTest(wal) defer streaming.RecoverWALForTest() @@ -1649,7 +1648,6 @@ func TestProxy_ImportV2(t *testing.T) { err = node.sched.Start() assert.NoError(t, err) chMgr := NewMockChannelsMgr(t) - chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil) node.chMgr = chMgr // no such collection @@ -1686,7 +1684,6 @@ func TestProxy_ImportV2(t *testing.T) { // set partition name and with partition key chMgr = NewMockChannelsMgr(t) chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"ch0"}, nil) - chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil) node.chMgr = chMgr rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{CollectionName: "aaa", PartitionName: "bbb"}) assert.NoError(t, err) diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index 5f8669ca34..a8c035c05c 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -4660,7 +4660,6 @@ func TestProxy_Import(t *testing.T) { wal := mock_streaming.NewMockWALAccesser(t) b := mock_streaming.NewMockBroadcast(t) wal.EXPECT().Broadcast().Return(b).Maybe() - b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() streaming.SetWALForTest(wal) defer streaming.RecoverWALForTest() @@ -4689,7 +4688,6 @@ func TestProxy_Import(t *testing.T) { chMgr := NewMockChannelsMgr(t) chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"foo"}, nil) - chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"foo_v1"}, nil) proxy.chMgr = chMgr factory := dependency.NewDefaultFactory(true) @@ -4716,7 +4714,6 @@ func TestProxy_Import(t *testing.T) { b := mock_streaming.NewMockBroadcast(t) wal.EXPECT().Broadcast().Return(b) b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil) - b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() streaming.SetWALForTest(wal) defer streaming.RecoverWALForTest() diff --git a/internal/proxy/task_import.go b/internal/proxy/task_import.go index 214abe8c0a..ace822612c 100644 --- a/internal/proxy/task_import.go +++ b/internal/proxy/task_import.go @@ -199,20 +199,13 @@ func (it *importTask) PreExecute(ctx context.Context) error { } func (it *importTask) setChannels() error { - collID, err := globalMetaCache.GetCollectionID(it.ctx, it.req.GetDbName(), it.req.CollectionName) - if err != nil { - return err - } - channels, err := it.node.chMgr.getChannels(collID) - if err != nil { - return err - } - it.pchannels = channels + // import task only send message by broadcast, which didn't affect the time tick rule at proxy. + // so we don't need to set channels here. return nil } func (it *importTask) getChannels() []pChan { - return it.pchannels + return nil } func (it *importTask) Execute(ctx context.Context) error { diff --git a/internal/rootcoord/broadcast_task.go b/internal/rootcoord/broadcast_task.go index d4f739182f..3a4438a7a2 100644 --- a/internal/rootcoord/broadcast_task.go +++ b/internal/rootcoord/broadcast_task.go @@ -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 diff --git a/internal/streamingcoord/client/broadcast/broadcast_impl.go b/internal/streamingcoord/client/broadcast/broadcast_impl.go index 640a6945c0..055fa6a60b 100644 --- a/internal/streamingcoord/client/broadcast/broadcast_impl.go +++ b/internal/streamingcoord/client/broadcast/broadcast_impl.go @@ -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() } diff --git a/internal/streamingcoord/client/broadcast/broadcast_test.go b/internal/streamingcoord/client/broadcast/broadcast_test.go index 518a76dccb..6fb5def29f 100644 --- a/internal/streamingcoord/client/broadcast/broadcast_test.go +++ b/internal/streamingcoord/client/broadcast/broadcast_test.go @@ -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 +} diff --git a/internal/streamingcoord/client/broadcast/grpc_watcher.go b/internal/streamingcoord/client/broadcast/grpc_watcher.go deleted file mode 100644 index 273c43637c..0000000000 --- a/internal/streamingcoord/client/broadcast/grpc_watcher.go +++ /dev/null @@ -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)) - } - } -} diff --git a/internal/streamingcoord/client/broadcast/grpc_watcher_test.go b/internal/streamingcoord/client/broadcast/grpc_watcher_test.go deleted file mode 100644 index 8e540c1a80..0000000000 --- a/internal/streamingcoord/client/broadcast/grpc_watcher_test.go +++ /dev/null @@ -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 -} diff --git a/internal/streamingcoord/client/broadcast/watcher.go b/internal/streamingcoord/client/broadcast/watcher.go deleted file mode 100644 index 5a69eaabf6..0000000000 --- a/internal/streamingcoord/client/broadcast/watcher.go +++ /dev/null @@ -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() -} diff --git a/internal/streamingcoord/client/broadcast/watcher_resuming.go b/internal/streamingcoord/client/broadcast/watcher_resuming.go deleted file mode 100644 index 0e6d699f7e..0000000000 --- a/internal/streamingcoord/client/broadcast/watcher_resuming.go +++ /dev/null @@ -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{} -} diff --git a/internal/streamingcoord/client/broadcast/watcher_resuming_test.go b/internal/streamingcoord/client/broadcast/watcher_resuming_test.go deleted file mode 100644 index 3c6dbdc177..0000000000 --- a/internal/streamingcoord/client/broadcast/watcher_resuming_test.go +++ /dev/null @@ -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 - }} -} diff --git a/internal/streamingcoord/client/client.go b/internal/streamingcoord/client/client.go index e2d8f2dff8..5a5ed46380 100644 --- a/internal/streamingcoord/client/client.go +++ b/internal/streamingcoord/client/client.go @@ -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() } diff --git a/internal/streamingcoord/server/broadcaster/broadcast_manager.go b/internal/streamingcoord/server/broadcaster/broadcast_manager.go index 20d88a5c8f..dca15fa7d7 100644 --- a/internal/streamingcoord/server/broadcaster/broadcast_manager.go +++ b/internal/streamingcoord/server/broadcaster/broadcast_manager.go @@ -9,7 +9,6 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" - "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" @@ -72,13 +71,14 @@ func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.Broadca if msg, err = bm.assignID(ctx, msg); err != nil { return nil, err } - task, err := bm.addBroadcastTask(msg) + task, err := bm.addBroadcastTask(ctx, msg) if err != nil { return nil, err } return newPendingBroadcastTask(task), nil } +// assignID assigns the broadcast id to the message. func (bm *broadcastTaskManager) assignID(ctx context.Context, msg message.BroadcastMutableMessage) (message.BroadcastMutableMessage, error) { // TODO: current implementation the header cannot be seen at flusher itself. // only import message use it, so temporarily set the broadcast id here. @@ -116,72 +116,56 @@ func (bm *broadcastTaskManager) Ack(ctx context.Context, broadcastID uint64, vch if task.State() == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { bm.removeBroadcastTask(broadcastID) - } else { - bm.increaseVersion() } return nil } -// WatchAtVersion watches the version of the broadcast task manager. -// When the version is greater than the input version, the watcher will be notified. -func (bm *broadcastTaskManager) WatchAtVersion(version int) <-chan struct{} { - bm.cond.L.Lock() - if bm.version > version { - bm.cond.L.Unlock() - ch := make(chan struct{}) - close(ch) - return ch - } - return bm.cond.WaitChan() -} - -// CurrentVersion returns the current version of the broadcast task manager. -func (bm *broadcastTaskManager) CurrentVersion() int { - bm.cond.L.Lock() - defer bm.cond.L.Unlock() - return bm.version -} - -// GetBroadcastTaskByResourceKey returns the broadcast task by the resource key. -func (bm *broadcastTaskManager) GetBroadcastTaskByResourceKey(resourceKey message.ResourceKey) (*broadcastTask, bool) { - bm.cond.L.Lock() +// ReleaseResourceKeys releases the resource keys by the broadcastID. +func (bm *broadcastTaskManager) ReleaseResourceKeys(broadcastID uint64) { + bm.cond.LockAndBroadcast() defer bm.cond.L.Unlock() - broadcastID, ok := bm.resourceKeys[resourceKey] - if !ok { - return nil, false - } - task, ok := bm.tasks[broadcastID] - return task, ok + bm.removeResourceKeys(broadcastID) } // addBroadcastTask adds the broadcast task into the manager. -func (bm *broadcastTaskManager) addBroadcastTask(msg message.BroadcastMutableMessage) (*broadcastTask, error) { +func (bm *broadcastTaskManager) addBroadcastTask(ctx context.Context, msg message.BroadcastMutableMessage) (*broadcastTask, error) { newIncomingTask := newBroadcastTaskFromBroadcastMessage(msg, bm.metrics) header := newIncomingTask.Header() newIncomingTask.SetLogger(bm.Logger().With(zap.Uint64("broadcastID", header.BroadcastID))) bm.cond.L.Lock() - defer bm.cond.L.Unlock() - // Check if the resource key is held by other task. - for key := range header.ResourceKeys { - if _, ok := bm.resourceKeys[key]; ok { - return nil, status.NewResourceAcquired(fmt.Sprintf("domain: %s, key: %s", key.Domain.String(), key.Key)) + for bm.checkIfResourceKeyExist(header) { + if err := bm.cond.Wait(ctx); err != nil { + return nil, err } } + // setup the resource keys to make resource exclusive held. for key := range header.ResourceKeys { bm.resourceKeys[key] = header.BroadcastID bm.metrics.IncomingResourceKey(key.Domain) } bm.tasks[header.BroadcastID] = newIncomingTask + bm.cond.L.Unlock() + // TODO: perform a task checker here to make sure the task is vaild to be broadcasted in future. return newIncomingTask, nil } +func (bm *broadcastTaskManager) checkIfResourceKeyExist(header *message.BroadcastHeader) bool { + for key := range header.ResourceKeys { + if _, ok := bm.resourceKeys[key]; ok { + return true + } + } + return false +} + // getBroadcastTaskByID return the task by the broadcastID. func (bm *broadcastTaskManager) getBroadcastTaskByID(broadcastID uint64) (*broadcastTask, bool) { bm.cond.L.Lock() defer bm.cond.L.Unlock() + t, ok := bm.tasks[broadcastID] return t, ok } @@ -191,21 +175,19 @@ func (bm *broadcastTaskManager) removeBroadcastTask(broadcastID uint64) { bm.cond.LockAndBroadcast() defer bm.cond.L.Unlock() + bm.removeResourceKeys(broadcastID) + delete(bm.tasks, broadcastID) +} + +// removeResourceKeys removes the resource keys by the broadcastID. +func (bm *broadcastTaskManager) removeResourceKeys(broadcastID uint64) { task, ok := bm.tasks[broadcastID] if !ok { return } - bm.version++ // remove the related resource keys for key := range task.header.ResourceKeys { delete(bm.resourceKeys, key) bm.metrics.GoneResourceKey(key.Domain) } - delete(bm.tasks, broadcastID) -} - -func (bm *broadcastTaskManager) increaseVersion() { - bm.cond.LockAndBroadcast() - bm.version++ - bm.cond.L.Unlock() } diff --git a/internal/streamingcoord/server/broadcaster/broadcast_task.go b/internal/streamingcoord/server/broadcaster/broadcast_task.go index 906ada650e..14fb3f21b0 100644 --- a/internal/streamingcoord/server/broadcaster/broadcast_task.go +++ b/internal/streamingcoord/server/broadcaster/broadcast_task.go @@ -20,20 +20,25 @@ func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask, metrics *broadc m := metrics.NewBroadcastTask(proto.GetState()) msg := message.NewBroadcastMutableMessageBeforeAppend(proto.Message.Payload, proto.Message.Properties) bh := msg.BroadcastHeader() - return &broadcastTask{ + bt := &broadcastTask{ mu: sync.Mutex{}, header: bh, task: proto, recoverPersisted: true, // the task is recovered from the recovery info, so it's persisted. metrics: m, + allAcked: make(chan struct{}), } + if isAllDone(proto) { + close(bt.allAcked) + } + return bt } // newBroadcastTaskFromBroadcastMessage creates a new broadcast task from the broadcast message. func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage, metrics *broadcasterMetrics) *broadcastTask { m := metrics.NewBroadcastTask(streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING) header := msg.BroadcastHeader() - return &broadcastTask{ + bt := &broadcastTask{ Binder: log.Binder{}, mu: sync.Mutex{}, header: header, @@ -44,7 +49,12 @@ func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage, m }, recoverPersisted: false, metrics: m, + allAcked: make(chan struct{}), } + if isAllDone(bt.task) { + close(bt.allAcked) + } + return bt } // broadcastTask is the state of the broadcast task. @@ -55,6 +65,7 @@ type broadcastTask struct { task *streamingpb.BroadcastTask recoverPersisted bool // a flag to indicate that the task has been persisted into the recovery info and can be recovered. metrics *taskMetricsGuard + allAcked chan struct{} } // Header returns the header of the broadcast task. @@ -121,13 +132,23 @@ func (b *broadcastTask) Ack(ctx context.Context, vchannel string) error { return err } b.task = task - b.metrics.ObserveAckAnyOne() if isAllDone(task) { b.metrics.ObserveAckAll() + close(b.allAcked) } return nil } +// BlockUntilAllAck blocks until all the vchannels are acked. +func (b *broadcastTask) BlockUntilAllAck(ctx context.Context) error { + select { + case <-ctx.Done(): + return ctx.Err() + case <-b.allAcked: + return nil + } +} + // copyAndSetVChannelAcked copies the task and set the vchannel as acked. // if the vchannel is already acked, it returns nil and false. func (b *broadcastTask) copyAndSetVChannelAcked(vchannel string) (*streamingpb.BroadcastTask, bool) { @@ -185,13 +206,6 @@ func (b *broadcastTask) copyAndMarkBroadcastDone() *streamingpb.BroadcastTask { return task } -// IsAllAcked returns true if all the vchannels are acked. -func (b *broadcastTask) IsAllAcked() bool { - b.mu.Lock() - defer b.mu.Unlock() - return isAllDone(b.task) -} - // isAllDone check if all the vchannels are acked. func isAllDone(task *streamingpb.BroadcastTask) bool { for _, acked := range task.AckedVchannelBitmap { @@ -211,18 +225,6 @@ func ackedCount(task *streamingpb.BroadcastTask) int { return count } -// IsAcked returns true if any vchannel is acked. -func (b *broadcastTask) IsAcked() bool { - b.mu.Lock() - defer b.mu.Unlock() - for _, acked := range b.task.AckedVchannelBitmap { - if acked != 0 { - return true - } - } - return false -} - // saveTask saves the broadcast task recovery info. func (b *broadcastTask) saveTask(ctx context.Context, task *streamingpb.BroadcastTask, logger *log.MLogger) error { logger = logger.With(zap.String("state", task.State.String()), zap.Int("ackedVChannelCount", ackedCount(task))) diff --git a/internal/streamingcoord/server/broadcaster/broadcaster.go b/internal/streamingcoord/server/broadcaster/broadcaster.go index c1328f7c20..a5221cb005 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster.go @@ -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 diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go index 63ea16268c..85480f257e 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go @@ -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) } } } diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_test.go b/internal/streamingcoord/server/broadcaster/broadcaster_test.go index e253401a68..cee5d151d3 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_test.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_test.go @@ -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 } diff --git a/internal/streamingcoord/server/broadcaster/metrics.go b/internal/streamingcoord/server/broadcaster/metrics.go index 134c2227cd..cb64fab6d5 100644 --- a/internal/streamingcoord/server/broadcaster/metrics.go +++ b/internal/streamingcoord/server/broadcaster/metrics.go @@ -20,7 +20,6 @@ func newBroadcasterMetrics() *broadcasterMetrics { taskTotal: metrics.StreamingCoordBroadcasterTaskTotal.MustCurryWith(constLabel), resourceKeyTotal: metrics.StreamingCoordResourceKeyTotal.MustCurryWith(constLabel), broadcastDuration: metrics.StreamingCoordBroadcastDurationSeconds.With(constLabel), - ackAnyOneDuration: metrics.StreamingCoordBroadcasterAckAnyOneDurationSeconds.With(constLabel), ackAllDuration: metrics.StreamingCoordBroadcasterAckAllDurationSeconds.With(constLabel), } } @@ -30,7 +29,6 @@ type broadcasterMetrics struct { taskTotal *prometheus.GaugeVec resourceKeyTotal *prometheus.GaugeVec broadcastDuration prometheus.Observer - ackAnyOneDuration prometheus.Observer ackAllDuration prometheus.Observer } @@ -80,11 +78,6 @@ func (g *taskMetricsGuard) ObserveBroadcastDone() { g.broadcastDuration.Observe(time.Since(g.start).Seconds()) } -// ObserverAckOne observes the ack any one. -func (g *taskMetricsGuard) ObserveAckAnyOne() { - g.ackAnyOneDuration.Observe(time.Since(g.start).Seconds()) -} - // ObserverAckOne observes the ack all. func (g *taskMetricsGuard) ObserveAckAll() { g.ackAllDuration.Observe(time.Since(g.start).Seconds()) diff --git a/internal/streamingcoord/server/broadcaster/watcher.go b/internal/streamingcoord/server/broadcaster/watcher.go deleted file mode 100644 index 02b5ccf1b0..0000000000 --- a/internal/streamingcoord/server/broadcaster/watcher.go +++ /dev/null @@ -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() -} diff --git a/internal/streamingcoord/server/service/broadcast.go b/internal/streamingcoord/server/service/broadcast.go index 1e5d9a2175..b6359c1b71 100644 --- a/internal/streamingcoord/server/service/broadcast.go +++ b/internal/streamingcoord/server/service/broadcast.go @@ -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() -} diff --git a/internal/streamingcoord/server/service/broadcast/broadcast_watch_grpc_server_helper.go b/internal/streamingcoord/server/service/broadcast/broadcast_watch_grpc_server_helper.go deleted file mode 100644 index dd3584e452..0000000000 --- a/internal/streamingcoord/server/service/broadcast/broadcast_watch_grpc_server_helper.go +++ /dev/null @@ -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{}, - }, - }) -} diff --git a/internal/streamingcoord/server/service/broadcast/broadcast_watch_server.go b/internal/streamingcoord/server/service/broadcast/broadcast_watch_server.go deleted file mode 100644 index 5886853f8d..0000000000 --- a/internal/streamingcoord/server/service/broadcast/broadcast_watch_server.go +++ /dev/null @@ -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 - } - } -} diff --git a/internal/streamingcoord/server/service/broadcast/broadcast_watch_test.go b/internal/streamingcoord/server/service/broadcast/broadcast_watch_test.go deleted file mode 100644 index 53c93aa498..0000000000 --- a/internal/streamingcoord/server/service/broadcast/broadcast_watch_test.go +++ /dev/null @@ -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 -} diff --git a/pkg/metrics/streaming_service_metrics.go b/pkg/metrics/streaming_service_metrics.go index 2f1962329a..3315959bc2 100644 --- a/pkg/metrics/streaming_service_metrics.go +++ b/pkg/metrics/streaming_service_metrics.go @@ -128,12 +128,6 @@ var ( Buckets: secondsBuckets, }) - StreamingCoordBroadcasterAckAnyOneDurationSeconds = newStreamingCoordHistogramVec(prometheus.HistogramOpts{ - Name: "broadcaster_ack_any_one_duration_seconds", - Help: "Duration of acknowledge any message", - Buckets: secondsBuckets, - }) - StreamingCoordBroadcasterAckAllDurationSeconds = newStreamingCoordHistogramVec(prometheus.HistogramOpts{ Name: "broadcaster_ack_all_duration_seconds", Help: "Duration of acknowledge all message", @@ -406,7 +400,6 @@ func registerStreamingCoord(registry *prometheus.Registry) { registry.MustRegister(StreamingCoordAssignmentListenerTotal) registry.MustRegister(StreamingCoordBroadcasterTaskTotal) registry.MustRegister(StreamingCoordBroadcastDurationSeconds) - registry.MustRegister(StreamingCoordBroadcasterAckAnyOneDurationSeconds) registry.MustRegister(StreamingCoordBroadcasterAckAllDurationSeconds) registry.MustRegister(StreamingCoordResourceKeyTotal) } diff --git a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go index 03582651d8..a475026d41 100644 --- a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go +++ b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go @@ -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 { diff --git a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchClient.go b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchClient.go deleted file mode 100644 index eed5a93cbb..0000000000 --- a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchClient.go +++ /dev/null @@ -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 -} diff --git a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchServer.go b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchServer.go deleted file mode 100644 index a9d64d3982..0000000000 --- a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchServer.go +++ /dev/null @@ -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 -} diff --git a/pkg/mq/msgstream/msg_for_import.go b/pkg/mq/msgstream/msg_for_import.go index 0c6ba59d0f..e3aefda93b 100644 --- a/pkg/mq/msgstream/msg_for_import.go +++ b/pkg/mq/msgstream/msg_for_import.go @@ -39,6 +39,11 @@ func (i *ImportMsg) SetID(id UniqueID) { i.Base.MsgID = id } +func (i *ImportMsg) SetTs(ts Timestamp) { + i.Base.Timestamp = ts + i.BaseMsg.SetTs(ts) +} + func (i *ImportMsg) Type() MsgType { return i.Base.MsgType } diff --git a/pkg/mq/msgstream/msg_for_import_test.go b/pkg/mq/msgstream/msg_for_import_test.go index e987f9011c..57dcfe6abe 100644 --- a/pkg/mq/msgstream/msg_for_import_test.go +++ b/pkg/mq/msgstream/msg_for_import_test.go @@ -61,4 +61,17 @@ func TestImportMsg(t *testing.T) { assert.EqualValues(t, 1000, newMsg.EndTs()) assert.True(t, msg.Size() > 0) + + msg.SetTs(3000) + assert.EqualValues(t, 3000, msg.BeginTs()) + assert.EqualValues(t, 3000, msg.EndTs()) + assert.EqualValues(t, 3000, msg.(*ImportMsg).Base.Timestamp) + + msgBytes, err = msg.Marshal(msg) + assert.NoError(t, err) + newMsg, err = newMsg.Unmarshal(msgBytes) + assert.NoError(t, err) + assert.EqualValues(t, 3000, newMsg.BeginTs()) + assert.EqualValues(t, 3000, newMsg.EndTs()) + assert.EqualValues(t, 3000, newMsg.(*ImportMsg).Base.Timestamp) } diff --git a/pkg/proto/streaming.proto b/pkg/proto/streaming.proto index 0b889363e4..79b1123d4e 100644 --- a/pkg/proto/streaming.proto +++ b/pkg/proto/streaming.proto @@ -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 // diff --git a/pkg/proto/streamingpb/streaming.pb.go b/pkg/proto/streamingpb/streaming.pb.go index 65045e7b77..5d75bbc359 100644 --- a/pkg/proto/streamingpb/streaming.pb.go +++ b/pkg/proto/streamingpb/streaming.pb.go @@ -779,338 +779,6 @@ func (*BroadcastAckResponse) Descriptor() ([]byte, []int) { return file_streaming_proto_rawDescGZIP(), []int{8} } -type BroadcastWatchRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Command: - // - // *BroadcastWatchRequest_CreateEventWatch - // *BroadcastWatchRequest_Close - Command isBroadcastWatchRequest_Command `protobuf_oneof:"command"` -} - -func (x *BroadcastWatchRequest) Reset() { - *x = BroadcastWatchRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BroadcastWatchRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BroadcastWatchRequest) ProtoMessage() {} - -func (x *BroadcastWatchRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[9] - 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 BroadcastWatchRequest.ProtoReflect.Descriptor instead. -func (*BroadcastWatchRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{9} -} - -func (m *BroadcastWatchRequest) GetCommand() isBroadcastWatchRequest_Command { - if m != nil { - return m.Command - } - return nil -} - -func (x *BroadcastWatchRequest) GetCreateEventWatch() *BroadcastCreateEventWatchRequest { - if x, ok := x.GetCommand().(*BroadcastWatchRequest_CreateEventWatch); ok { - return x.CreateEventWatch - } - return nil -} - -func (x *BroadcastWatchRequest) GetClose() *CloseBroadcastWatchRequest { - if x, ok := x.GetCommand().(*BroadcastWatchRequest_Close); ok { - return x.Close - } - return nil -} - -type isBroadcastWatchRequest_Command interface { - isBroadcastWatchRequest_Command() -} - -type BroadcastWatchRequest_CreateEventWatch struct { - CreateEventWatch *BroadcastCreateEventWatchRequest `protobuf:"bytes,1,opt,name=create_event_watch,json=createEventWatch,proto3,oneof"` -} - -type BroadcastWatchRequest_Close struct { - Close *CloseBroadcastWatchRequest `protobuf:"bytes,2,opt,name=close,proto3,oneof"` -} - -func (*BroadcastWatchRequest_CreateEventWatch) isBroadcastWatchRequest_Command() {} - -func (*BroadcastWatchRequest_Close) isBroadcastWatchRequest_Command() {} - -type BroadcastCreateEventWatchRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Event *messagespb.BroadcastEvent `protobuf:"bytes,1,opt,name=event,proto3" json:"event,omitempty"` -} - -func (x *BroadcastCreateEventWatchRequest) Reset() { - *x = BroadcastCreateEventWatchRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BroadcastCreateEventWatchRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BroadcastCreateEventWatchRequest) ProtoMessage() {} - -func (x *BroadcastCreateEventWatchRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[10] - 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 BroadcastCreateEventWatchRequest.ProtoReflect.Descriptor instead. -func (*BroadcastCreateEventWatchRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{10} -} - -func (x *BroadcastCreateEventWatchRequest) GetEvent() *messagespb.BroadcastEvent { - if x != nil { - return x.Event - } - return nil -} - -type CloseBroadcastWatchRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields -} - -func (x *CloseBroadcastWatchRequest) Reset() { - *x = CloseBroadcastWatchRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *CloseBroadcastWatchRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*CloseBroadcastWatchRequest) ProtoMessage() {} - -func (x *CloseBroadcastWatchRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[11] - 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 CloseBroadcastWatchRequest.ProtoReflect.Descriptor instead. -func (*CloseBroadcastWatchRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{11} -} - -type BroadcastWatchResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Response: - // - // *BroadcastWatchResponse_EventDone - // *BroadcastWatchResponse_Close - Response isBroadcastWatchResponse_Response `protobuf_oneof:"response"` -} - -func (x *BroadcastWatchResponse) Reset() { - *x = BroadcastWatchResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BroadcastWatchResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BroadcastWatchResponse) ProtoMessage() {} - -func (x *BroadcastWatchResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[12] - 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 BroadcastWatchResponse.ProtoReflect.Descriptor instead. -func (*BroadcastWatchResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{12} -} - -func (m *BroadcastWatchResponse) GetResponse() isBroadcastWatchResponse_Response { - if m != nil { - return m.Response - } - return nil -} - -func (x *BroadcastWatchResponse) GetEventDone() *BroadcastEventWatchResponse { - if x, ok := x.GetResponse().(*BroadcastWatchResponse_EventDone); ok { - return x.EventDone - } - return nil -} - -func (x *BroadcastWatchResponse) GetClose() *CloseBroadcastWatchResponse { - if x, ok := x.GetResponse().(*BroadcastWatchResponse_Close); ok { - return x.Close - } - return nil -} - -type isBroadcastWatchResponse_Response interface { - isBroadcastWatchResponse_Response() -} - -type BroadcastWatchResponse_EventDone struct { - EventDone *BroadcastEventWatchResponse `protobuf:"bytes,1,opt,name=event_done,json=eventDone,proto3,oneof"` -} - -type BroadcastWatchResponse_Close struct { - Close *CloseBroadcastWatchResponse `protobuf:"bytes,2,opt,name=close,proto3,oneof"` -} - -func (*BroadcastWatchResponse_EventDone) isBroadcastWatchResponse_Response() {} - -func (*BroadcastWatchResponse_Close) isBroadcastWatchResponse_Response() {} - -type BroadcastEventWatchResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Event *messagespb.BroadcastEvent `protobuf:"bytes,1,opt,name=event,proto3" json:"event,omitempty"` -} - -func (x *BroadcastEventWatchResponse) Reset() { - *x = BroadcastEventWatchResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *BroadcastEventWatchResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*BroadcastEventWatchResponse) ProtoMessage() {} - -func (x *BroadcastEventWatchResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[13] - 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 BroadcastEventWatchResponse.ProtoReflect.Descriptor instead. -func (*BroadcastEventWatchResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{13} -} - -func (x *BroadcastEventWatchResponse) GetEvent() *messagespb.BroadcastEvent { - if x != nil { - return x.Event - } - return nil -} - -type CloseBroadcastWatchResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields -} - -func (x *CloseBroadcastWatchResponse) Reset() { - *x = CloseBroadcastWatchResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *CloseBroadcastWatchResponse) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*CloseBroadcastWatchResponse) ProtoMessage() {} - -func (x *CloseBroadcastWatchResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[14] - 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 CloseBroadcastWatchResponse.ProtoReflect.Descriptor instead. -func (*CloseBroadcastWatchResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{14} -} - // AssignmentDiscoverRequest is the request of Discovery type AssignmentDiscoverRequest struct { state protoimpl.MessageState @@ -1127,7 +795,7 @@ type AssignmentDiscoverRequest struct { func (x *AssignmentDiscoverRequest) Reset() { *x = AssignmentDiscoverRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[15] + mi := &file_streaming_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1140,7 +808,7 @@ func (x *AssignmentDiscoverRequest) String() string { func (*AssignmentDiscoverRequest) ProtoMessage() {} func (x *AssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[15] + mi := &file_streaming_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1153,7 +821,7 @@ func (x *AssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignmentDiscoverRequest.ProtoReflect.Descriptor instead. func (*AssignmentDiscoverRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{15} + return file_streaming_proto_rawDescGZIP(), []int{9} } func (m *AssignmentDiscoverRequest) GetCommand() isAssignmentDiscoverRequest_Command { @@ -1207,7 +875,7 @@ type ReportAssignmentErrorRequest struct { func (x *ReportAssignmentErrorRequest) Reset() { *x = ReportAssignmentErrorRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[16] + mi := &file_streaming_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1220,7 +888,7 @@ func (x *ReportAssignmentErrorRequest) String() string { func (*ReportAssignmentErrorRequest) ProtoMessage() {} func (x *ReportAssignmentErrorRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[16] + mi := &file_streaming_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1233,7 +901,7 @@ func (x *ReportAssignmentErrorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReportAssignmentErrorRequest.ProtoReflect.Descriptor instead. func (*ReportAssignmentErrorRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{16} + return file_streaming_proto_rawDescGZIP(), []int{10} } func (x *ReportAssignmentErrorRequest) GetPchannel() *PChannelInfo { @@ -1260,7 +928,7 @@ type CloseAssignmentDiscoverRequest struct { func (x *CloseAssignmentDiscoverRequest) Reset() { *x = CloseAssignmentDiscoverRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[17] + mi := &file_streaming_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1273,7 +941,7 @@ func (x *CloseAssignmentDiscoverRequest) String() string { func (*CloseAssignmentDiscoverRequest) ProtoMessage() {} func (x *CloseAssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[17] + mi := &file_streaming_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1286,7 +954,7 @@ func (x *CloseAssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseAssignmentDiscoverRequest.ProtoReflect.Descriptor instead. func (*CloseAssignmentDiscoverRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{17} + return file_streaming_proto_rawDescGZIP(), []int{11} } // AssignmentDiscoverResponse is the response of Discovery @@ -1305,7 +973,7 @@ type AssignmentDiscoverResponse struct { func (x *AssignmentDiscoverResponse) Reset() { *x = AssignmentDiscoverResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[18] + mi := &file_streaming_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1318,7 +986,7 @@ func (x *AssignmentDiscoverResponse) String() string { func (*AssignmentDiscoverResponse) ProtoMessage() {} func (x *AssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[18] + mi := &file_streaming_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1331,7 +999,7 @@ func (x *AssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignmentDiscoverResponse.ProtoReflect.Descriptor instead. func (*AssignmentDiscoverResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{18} + return file_streaming_proto_rawDescGZIP(), []int{12} } func (m *AssignmentDiscoverResponse) GetResponse() isAssignmentDiscoverResponse_Response { @@ -1386,7 +1054,7 @@ type FullStreamingNodeAssignmentWithVersion struct { func (x *FullStreamingNodeAssignmentWithVersion) Reset() { *x = FullStreamingNodeAssignmentWithVersion{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[19] + mi := &file_streaming_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1399,7 +1067,7 @@ func (x *FullStreamingNodeAssignmentWithVersion) String() string { func (*FullStreamingNodeAssignmentWithVersion) ProtoMessage() {} func (x *FullStreamingNodeAssignmentWithVersion) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[19] + mi := &file_streaming_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1412,7 +1080,7 @@ func (x *FullStreamingNodeAssignmentWithVersion) ProtoReflect() protoreflect.Mes // Deprecated: Use FullStreamingNodeAssignmentWithVersion.ProtoReflect.Descriptor instead. func (*FullStreamingNodeAssignmentWithVersion) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{19} + return file_streaming_proto_rawDescGZIP(), []int{13} } func (x *FullStreamingNodeAssignmentWithVersion) GetVersion() *VersionPair { @@ -1438,7 +1106,7 @@ type CloseAssignmentDiscoverResponse struct { func (x *CloseAssignmentDiscoverResponse) Reset() { *x = CloseAssignmentDiscoverResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[20] + mi := &file_streaming_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1451,7 +1119,7 @@ func (x *CloseAssignmentDiscoverResponse) String() string { func (*CloseAssignmentDiscoverResponse) ProtoMessage() {} func (x *CloseAssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[20] + mi := &file_streaming_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1464,7 +1132,7 @@ func (x *CloseAssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseAssignmentDiscoverResponse.ProtoReflect.Descriptor instead. func (*CloseAssignmentDiscoverResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{20} + return file_streaming_proto_rawDescGZIP(), []int{14} } // StreamingNodeInfo is the information of a streaming node. @@ -1480,7 +1148,7 @@ type StreamingNodeInfo struct { func (x *StreamingNodeInfo) Reset() { *x = StreamingNodeInfo{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[21] + mi := &file_streaming_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1493,7 +1161,7 @@ func (x *StreamingNodeInfo) String() string { func (*StreamingNodeInfo) ProtoMessage() {} func (x *StreamingNodeInfo) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[21] + mi := &file_streaming_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1506,7 +1174,7 @@ func (x *StreamingNodeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingNodeInfo.ProtoReflect.Descriptor instead. func (*StreamingNodeInfo) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{21} + return file_streaming_proto_rawDescGZIP(), []int{15} } func (x *StreamingNodeInfo) GetServerId() int64 { @@ -1536,7 +1204,7 @@ type StreamingNodeAssignment struct { func (x *StreamingNodeAssignment) Reset() { *x = StreamingNodeAssignment{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[22] + mi := &file_streaming_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1549,7 +1217,7 @@ func (x *StreamingNodeAssignment) String() string { func (*StreamingNodeAssignment) ProtoMessage() {} func (x *StreamingNodeAssignment) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[22] + mi := &file_streaming_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1562,7 +1230,7 @@ func (x *StreamingNodeAssignment) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingNodeAssignment.ProtoReflect.Descriptor instead. func (*StreamingNodeAssignment) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{22} + return file_streaming_proto_rawDescGZIP(), []int{16} } func (x *StreamingNodeAssignment) GetNode() *StreamingNodeInfo { @@ -1597,7 +1265,7 @@ type DeliverPolicy struct { func (x *DeliverPolicy) Reset() { *x = DeliverPolicy{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[23] + mi := &file_streaming_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1610,7 +1278,7 @@ func (x *DeliverPolicy) String() string { func (*DeliverPolicy) ProtoMessage() {} func (x *DeliverPolicy) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[23] + mi := &file_streaming_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1623,7 +1291,7 @@ func (x *DeliverPolicy) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverPolicy.ProtoReflect.Descriptor instead. func (*DeliverPolicy) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{23} + return file_streaming_proto_rawDescGZIP(), []int{17} } func (m *DeliverPolicy) GetPolicy() isDeliverPolicy_Policy { @@ -1706,7 +1374,7 @@ type DeliverFilter struct { func (x *DeliverFilter) Reset() { *x = DeliverFilter{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[24] + mi := &file_streaming_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1719,7 +1387,7 @@ func (x *DeliverFilter) String() string { func (*DeliverFilter) ProtoMessage() {} func (x *DeliverFilter) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[24] + mi := &file_streaming_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1732,7 +1400,7 @@ func (x *DeliverFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilter.ProtoReflect.Descriptor instead. func (*DeliverFilter) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{24} + return file_streaming_proto_rawDescGZIP(), []int{18} } func (m *DeliverFilter) GetFilter() isDeliverFilter_Filter { @@ -1798,7 +1466,7 @@ type DeliverFilterTimeTickGT struct { func (x *DeliverFilterTimeTickGT) Reset() { *x = DeliverFilterTimeTickGT{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[25] + mi := &file_streaming_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1811,7 +1479,7 @@ func (x *DeliverFilterTimeTickGT) String() string { func (*DeliverFilterTimeTickGT) ProtoMessage() {} func (x *DeliverFilterTimeTickGT) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[25] + mi := &file_streaming_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1824,7 +1492,7 @@ func (x *DeliverFilterTimeTickGT) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilterTimeTickGT.ProtoReflect.Descriptor instead. func (*DeliverFilterTimeTickGT) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{25} + return file_streaming_proto_rawDescGZIP(), []int{19} } func (x *DeliverFilterTimeTickGT) GetTimeTick() uint64 { @@ -1847,7 +1515,7 @@ type DeliverFilterTimeTickGTE struct { func (x *DeliverFilterTimeTickGTE) Reset() { *x = DeliverFilterTimeTickGTE{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[26] + mi := &file_streaming_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1860,7 +1528,7 @@ func (x *DeliverFilterTimeTickGTE) String() string { func (*DeliverFilterTimeTickGTE) ProtoMessage() {} func (x *DeliverFilterTimeTickGTE) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[26] + mi := &file_streaming_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1873,7 +1541,7 @@ func (x *DeliverFilterTimeTickGTE) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilterTimeTickGTE.ProtoReflect.Descriptor instead. func (*DeliverFilterTimeTickGTE) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{26} + return file_streaming_proto_rawDescGZIP(), []int{20} } func (x *DeliverFilterTimeTickGTE) GetTimeTick() uint64 { @@ -1895,7 +1563,7 @@ type DeliverFilterMessageType struct { func (x *DeliverFilterMessageType) Reset() { *x = DeliverFilterMessageType{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[27] + mi := &file_streaming_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1908,7 +1576,7 @@ func (x *DeliverFilterMessageType) String() string { func (*DeliverFilterMessageType) ProtoMessage() {} func (x *DeliverFilterMessageType) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[27] + mi := &file_streaming_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1921,7 +1589,7 @@ func (x *DeliverFilterMessageType) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilterMessageType.ProtoReflect.Descriptor instead. func (*DeliverFilterMessageType) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{27} + return file_streaming_proto_rawDescGZIP(), []int{21} } func (x *DeliverFilterMessageType) GetMessageTypes() []messagespb.MessageType { @@ -1944,7 +1612,7 @@ type StreamingError struct { func (x *StreamingError) Reset() { *x = StreamingError{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[28] + mi := &file_streaming_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1957,7 +1625,7 @@ func (x *StreamingError) String() string { func (*StreamingError) ProtoMessage() {} func (x *StreamingError) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[28] + mi := &file_streaming_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1970,7 +1638,7 @@ func (x *StreamingError) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingError.ProtoReflect.Descriptor instead. func (*StreamingError) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{28} + return file_streaming_proto_rawDescGZIP(), []int{22} } func (x *StreamingError) GetCode() StreamingCode { @@ -2005,7 +1673,7 @@ type ProduceRequest struct { func (x *ProduceRequest) Reset() { *x = ProduceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[29] + mi := &file_streaming_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2018,7 +1686,7 @@ func (x *ProduceRequest) String() string { func (*ProduceRequest) ProtoMessage() {} func (x *ProduceRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[29] + mi := &file_streaming_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2031,7 +1699,7 @@ func (x *ProduceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceRequest.ProtoReflect.Descriptor instead. func (*ProduceRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{29} + return file_streaming_proto_rawDescGZIP(), []int{23} } func (m *ProduceRequest) GetRequest() isProduceRequest_Request { @@ -2084,7 +1752,7 @@ type CreateProducerRequest struct { func (x *CreateProducerRequest) Reset() { *x = CreateProducerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[30] + mi := &file_streaming_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2097,7 +1765,7 @@ func (x *CreateProducerRequest) String() string { func (*CreateProducerRequest) ProtoMessage() {} func (x *CreateProducerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[30] + mi := &file_streaming_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2110,7 +1778,7 @@ func (x *CreateProducerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateProducerRequest.ProtoReflect.Descriptor instead. func (*CreateProducerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{30} + return file_streaming_proto_rawDescGZIP(), []int{24} } func (x *CreateProducerRequest) GetPchannel() *PChannelInfo { @@ -2133,7 +1801,7 @@ type ProduceMessageRequest struct { func (x *ProduceMessageRequest) Reset() { *x = ProduceMessageRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[31] + mi := &file_streaming_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2146,7 +1814,7 @@ func (x *ProduceMessageRequest) String() string { func (*ProduceMessageRequest) ProtoMessage() {} func (x *ProduceMessageRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[31] + mi := &file_streaming_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2159,7 +1827,7 @@ func (x *ProduceMessageRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceMessageRequest.ProtoReflect.Descriptor instead. func (*ProduceMessageRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{31} + return file_streaming_proto_rawDescGZIP(), []int{25} } func (x *ProduceMessageRequest) GetRequestId() int64 { @@ -2187,7 +1855,7 @@ type CloseProducerRequest struct { func (x *CloseProducerRequest) Reset() { *x = CloseProducerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[32] + mi := &file_streaming_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2200,7 +1868,7 @@ func (x *CloseProducerRequest) String() string { func (*CloseProducerRequest) ProtoMessage() {} func (x *CloseProducerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[32] + mi := &file_streaming_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2213,7 +1881,7 @@ func (x *CloseProducerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseProducerRequest.ProtoReflect.Descriptor instead. func (*CloseProducerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{32} + return file_streaming_proto_rawDescGZIP(), []int{26} } // ProduceResponse is the response of the Produce RPC. @@ -2233,7 +1901,7 @@ type ProduceResponse struct { func (x *ProduceResponse) Reset() { *x = ProduceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[33] + mi := &file_streaming_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2246,7 +1914,7 @@ func (x *ProduceResponse) String() string { func (*ProduceResponse) ProtoMessage() {} func (x *ProduceResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[33] + mi := &file_streaming_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2259,7 +1927,7 @@ func (x *ProduceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceResponse.ProtoReflect.Descriptor instead. func (*ProduceResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{33} + return file_streaming_proto_rawDescGZIP(), []int{27} } func (m *ProduceResponse) GetResponse() isProduceResponse_Response { @@ -2325,7 +1993,7 @@ type CreateProducerResponse struct { func (x *CreateProducerResponse) Reset() { *x = CreateProducerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[34] + mi := &file_streaming_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2338,7 +2006,7 @@ func (x *CreateProducerResponse) String() string { func (*CreateProducerResponse) ProtoMessage() {} func (x *CreateProducerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[34] + mi := &file_streaming_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2351,7 +2019,7 @@ func (x *CreateProducerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateProducerResponse.ProtoReflect.Descriptor instead. func (*CreateProducerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{34} + return file_streaming_proto_rawDescGZIP(), []int{28} } func (x *CreateProducerResponse) GetWalName() string { @@ -2385,7 +2053,7 @@ type ProduceMessageResponse struct { func (x *ProduceMessageResponse) Reset() { *x = ProduceMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[35] + mi := &file_streaming_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2398,7 +2066,7 @@ func (x *ProduceMessageResponse) String() string { func (*ProduceMessageResponse) ProtoMessage() {} func (x *ProduceMessageResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[35] + mi := &file_streaming_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2411,7 +2079,7 @@ func (x *ProduceMessageResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceMessageResponse.ProtoReflect.Descriptor instead. func (*ProduceMessageResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{35} + return file_streaming_proto_rawDescGZIP(), []int{29} } func (x *ProduceMessageResponse) GetRequestId() int64 { @@ -2474,7 +2142,7 @@ type ProduceMessageResponseResult struct { func (x *ProduceMessageResponseResult) Reset() { *x = ProduceMessageResponseResult{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[36] + mi := &file_streaming_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2487,7 +2155,7 @@ func (x *ProduceMessageResponseResult) String() string { func (*ProduceMessageResponseResult) ProtoMessage() {} func (x *ProduceMessageResponseResult) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[36] + mi := &file_streaming_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2500,7 +2168,7 @@ func (x *ProduceMessageResponseResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceMessageResponseResult.ProtoReflect.Descriptor instead. func (*ProduceMessageResponseResult) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{36} + return file_streaming_proto_rawDescGZIP(), []int{30} } func (x *ProduceMessageResponseResult) GetId() *messagespb.MessageID { @@ -2541,7 +2209,7 @@ type CloseProducerResponse struct { func (x *CloseProducerResponse) Reset() { *x = CloseProducerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[37] + mi := &file_streaming_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2554,7 +2222,7 @@ func (x *CloseProducerResponse) String() string { func (*CloseProducerResponse) ProtoMessage() {} func (x *CloseProducerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[37] + mi := &file_streaming_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2567,7 +2235,7 @@ func (x *CloseProducerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseProducerResponse.ProtoReflect.Descriptor instead. func (*CloseProducerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{37} + return file_streaming_proto_rawDescGZIP(), []int{31} } // ConsumeRequest is the request of the Consume RPC. @@ -2589,7 +2257,7 @@ type ConsumeRequest struct { func (x *ConsumeRequest) Reset() { *x = ConsumeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[38] + mi := &file_streaming_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2602,7 +2270,7 @@ func (x *ConsumeRequest) String() string { func (*ConsumeRequest) ProtoMessage() {} func (x *ConsumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[38] + mi := &file_streaming_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2615,7 +2283,7 @@ func (x *ConsumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ConsumeRequest.ProtoReflect.Descriptor instead. func (*ConsumeRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{38} + return file_streaming_proto_rawDescGZIP(), []int{32} } func (m *ConsumeRequest) GetRequest() isConsumeRequest_Request { @@ -2692,7 +2360,7 @@ type CloseConsumerRequest struct { func (x *CloseConsumerRequest) Reset() { *x = CloseConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[39] + mi := &file_streaming_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2705,7 +2373,7 @@ func (x *CloseConsumerRequest) String() string { func (*CloseConsumerRequest) ProtoMessage() {} func (x *CloseConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[39] + mi := &file_streaming_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2718,7 +2386,7 @@ func (x *CloseConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseConsumerRequest.ProtoReflect.Descriptor instead. func (*CloseConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{39} + return file_streaming_proto_rawDescGZIP(), []int{33} } // CreateConsumerRequest is the request of the CreateConsumer RPC. @@ -2734,7 +2402,7 @@ type CreateConsumerRequest struct { func (x *CreateConsumerRequest) Reset() { *x = CreateConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[40] + mi := &file_streaming_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2747,7 +2415,7 @@ func (x *CreateConsumerRequest) String() string { func (*CreateConsumerRequest) ProtoMessage() {} func (x *CreateConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[40] + mi := &file_streaming_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2760,7 +2428,7 @@ func (x *CreateConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateConsumerRequest.ProtoReflect.Descriptor instead. func (*CreateConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{40} + return file_streaming_proto_rawDescGZIP(), []int{34} } func (x *CreateConsumerRequest) GetPchannel() *PChannelInfo { @@ -2781,7 +2449,7 @@ type CreateVChannelConsumersRequest struct { func (x *CreateVChannelConsumersRequest) Reset() { *x = CreateVChannelConsumersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[41] + mi := &file_streaming_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2794,7 +2462,7 @@ func (x *CreateVChannelConsumersRequest) String() string { func (*CreateVChannelConsumersRequest) ProtoMessage() {} func (x *CreateVChannelConsumersRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[41] + mi := &file_streaming_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2807,7 +2475,7 @@ func (x *CreateVChannelConsumersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumersRequest.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumersRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{41} + return file_streaming_proto_rawDescGZIP(), []int{35} } func (x *CreateVChannelConsumersRequest) GetCreateVchannels() []*CreateVChannelConsumerRequest { @@ -2832,7 +2500,7 @@ type CreateVChannelConsumerRequest struct { func (x *CreateVChannelConsumerRequest) Reset() { *x = CreateVChannelConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[42] + mi := &file_streaming_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2845,7 +2513,7 @@ func (x *CreateVChannelConsumerRequest) String() string { func (*CreateVChannelConsumerRequest) ProtoMessage() {} func (x *CreateVChannelConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[42] + mi := &file_streaming_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2858,7 +2526,7 @@ func (x *CreateVChannelConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumerRequest.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{42} + return file_streaming_proto_rawDescGZIP(), []int{36} } func (x *CreateVChannelConsumerRequest) GetVchannel() string { @@ -2894,7 +2562,7 @@ type CreateVChannelConsumersResponse struct { func (x *CreateVChannelConsumersResponse) Reset() { *x = CreateVChannelConsumersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[43] + mi := &file_streaming_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2907,7 +2575,7 @@ func (x *CreateVChannelConsumersResponse) String() string { func (*CreateVChannelConsumersResponse) ProtoMessage() {} func (x *CreateVChannelConsumersResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[43] + mi := &file_streaming_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2920,7 +2588,7 @@ func (x *CreateVChannelConsumersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumersResponse.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumersResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{43} + return file_streaming_proto_rawDescGZIP(), []int{37} } func (x *CreateVChannelConsumersResponse) GetCreateVchannels() []*CreateVChannelConsumerResponse { @@ -2947,7 +2615,7 @@ type CreateVChannelConsumerResponse struct { func (x *CreateVChannelConsumerResponse) Reset() { *x = CreateVChannelConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[44] + mi := &file_streaming_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2960,7 +2628,7 @@ func (x *CreateVChannelConsumerResponse) String() string { func (*CreateVChannelConsumerResponse) ProtoMessage() {} func (x *CreateVChannelConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[44] + mi := &file_streaming_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2973,7 +2641,7 @@ func (x *CreateVChannelConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumerResponse.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{44} + return file_streaming_proto_rawDescGZIP(), []int{38} } func (m *CreateVChannelConsumerResponse) GetResponse() isCreateVChannelConsumerResponse_Response { @@ -3025,7 +2693,7 @@ type CloseVChannelConsumerRequest struct { func (x *CloseVChannelConsumerRequest) Reset() { *x = CloseVChannelConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[45] + mi := &file_streaming_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3038,7 +2706,7 @@ func (x *CloseVChannelConsumerRequest) String() string { func (*CloseVChannelConsumerRequest) ProtoMessage() {} func (x *CloseVChannelConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[45] + mi := &file_streaming_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3051,7 +2719,7 @@ func (x *CloseVChannelConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseVChannelConsumerRequest.ProtoReflect.Descriptor instead. func (*CloseVChannelConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{45} + return file_streaming_proto_rawDescGZIP(), []int{39} } func (x *CloseVChannelConsumerRequest) GetConsumerId() int64 { @@ -3074,7 +2742,7 @@ type CloseVChannelConsumerResponse struct { func (x *CloseVChannelConsumerResponse) Reset() { *x = CloseVChannelConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[46] + mi := &file_streaming_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3087,7 +2755,7 @@ func (x *CloseVChannelConsumerResponse) String() string { func (*CloseVChannelConsumerResponse) ProtoMessage() {} func (x *CloseVChannelConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[46] + mi := &file_streaming_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3100,7 +2768,7 @@ func (x *CloseVChannelConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseVChannelConsumerResponse.ProtoReflect.Descriptor instead. func (*CloseVChannelConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{46} + return file_streaming_proto_rawDescGZIP(), []int{40} } func (x *CloseVChannelConsumerResponse) GetConsumerId() int64 { @@ -3130,7 +2798,7 @@ type ConsumeResponse struct { func (x *ConsumeResponse) Reset() { *x = ConsumeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[47] + mi := &file_streaming_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3143,7 +2811,7 @@ func (x *ConsumeResponse) String() string { func (*ConsumeResponse) ProtoMessage() {} func (x *ConsumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[47] + mi := &file_streaming_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3156,7 +2824,7 @@ func (x *ConsumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConsumeResponse.ProtoReflect.Descriptor instead. func (*ConsumeResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{47} + return file_streaming_proto_rawDescGZIP(), []int{41} } func (m *ConsumeResponse) GetResponse() isConsumeResponse_Response { @@ -3262,7 +2930,7 @@ type CreateConsumerResponse struct { func (x *CreateConsumerResponse) Reset() { *x = CreateConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[48] + mi := &file_streaming_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3275,7 +2943,7 @@ func (x *CreateConsumerResponse) String() string { func (*CreateConsumerResponse) ProtoMessage() {} func (x *CreateConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[48] + mi := &file_streaming_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3288,7 +2956,7 @@ func (x *CreateConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateConsumerResponse.ProtoReflect.Descriptor instead. func (*CreateConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{48} + return file_streaming_proto_rawDescGZIP(), []int{42} } func (x *CreateConsumerResponse) GetWalName() string { @@ -3317,7 +2985,7 @@ type ConsumeMessageReponse struct { func (x *ConsumeMessageReponse) Reset() { *x = ConsumeMessageReponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[49] + mi := &file_streaming_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3330,7 +2998,7 @@ func (x *ConsumeMessageReponse) String() string { func (*ConsumeMessageReponse) ProtoMessage() {} func (x *ConsumeMessageReponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[49] + mi := &file_streaming_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3343,7 +3011,7 @@ func (x *ConsumeMessageReponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConsumeMessageReponse.ProtoReflect.Descriptor instead. func (*ConsumeMessageReponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{49} + return file_streaming_proto_rawDescGZIP(), []int{43} } func (x *ConsumeMessageReponse) GetConsumerId() int64 { @@ -3369,7 +3037,7 @@ type CloseConsumerResponse struct { func (x *CloseConsumerResponse) Reset() { *x = CloseConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[50] + mi := &file_streaming_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3382,7 +3050,7 @@ func (x *CloseConsumerResponse) String() string { func (*CloseConsumerResponse) ProtoMessage() {} func (x *CloseConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[50] + mi := &file_streaming_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3395,7 +3063,7 @@ func (x *CloseConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseConsumerResponse.ProtoReflect.Descriptor instead. func (*CloseConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{50} + return file_streaming_proto_rawDescGZIP(), []int{44} } // StreamingManagerAssignRequest is the request message of Assign RPC. @@ -3410,7 +3078,7 @@ type StreamingNodeManagerAssignRequest struct { func (x *StreamingNodeManagerAssignRequest) Reset() { *x = StreamingNodeManagerAssignRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[51] + mi := &file_streaming_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3423,7 +3091,7 @@ func (x *StreamingNodeManagerAssignRequest) String() string { func (*StreamingNodeManagerAssignRequest) ProtoMessage() {} func (x *StreamingNodeManagerAssignRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[51] + mi := &file_streaming_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3436,7 +3104,7 @@ func (x *StreamingNodeManagerAssignRequest) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerAssignRequest.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerAssignRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{51} + return file_streaming_proto_rawDescGZIP(), []int{45} } func (x *StreamingNodeManagerAssignRequest) GetPchannel() *PChannelInfo { @@ -3455,7 +3123,7 @@ type StreamingNodeManagerAssignResponse struct { func (x *StreamingNodeManagerAssignResponse) Reset() { *x = StreamingNodeManagerAssignResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[52] + mi := &file_streaming_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3468,7 +3136,7 @@ func (x *StreamingNodeManagerAssignResponse) String() string { func (*StreamingNodeManagerAssignResponse) ProtoMessage() {} func (x *StreamingNodeManagerAssignResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[52] + mi := &file_streaming_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3481,7 +3149,7 @@ func (x *StreamingNodeManagerAssignResponse) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerAssignResponse.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerAssignResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{52} + return file_streaming_proto_rawDescGZIP(), []int{46} } type StreamingNodeManagerRemoveRequest struct { @@ -3495,7 +3163,7 @@ type StreamingNodeManagerRemoveRequest struct { func (x *StreamingNodeManagerRemoveRequest) Reset() { *x = StreamingNodeManagerRemoveRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[53] + mi := &file_streaming_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3508,7 +3176,7 @@ func (x *StreamingNodeManagerRemoveRequest) String() string { func (*StreamingNodeManagerRemoveRequest) ProtoMessage() {} func (x *StreamingNodeManagerRemoveRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[53] + mi := &file_streaming_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3521,7 +3189,7 @@ func (x *StreamingNodeManagerRemoveRequest) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerRemoveRequest.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerRemoveRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{53} + return file_streaming_proto_rawDescGZIP(), []int{47} } func (x *StreamingNodeManagerRemoveRequest) GetPchannel() *PChannelInfo { @@ -3540,7 +3208,7 @@ type StreamingNodeManagerRemoveResponse struct { func (x *StreamingNodeManagerRemoveResponse) Reset() { *x = StreamingNodeManagerRemoveResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[54] + mi := &file_streaming_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3553,7 +3221,7 @@ func (x *StreamingNodeManagerRemoveResponse) String() string { func (*StreamingNodeManagerRemoveResponse) ProtoMessage() {} func (x *StreamingNodeManagerRemoveResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[54] + mi := &file_streaming_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3566,7 +3234,7 @@ func (x *StreamingNodeManagerRemoveResponse) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerRemoveResponse.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerRemoveResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{54} + return file_streaming_proto_rawDescGZIP(), []int{48} } type StreamingNodeManagerCollectStatusRequest struct { @@ -3578,7 +3246,7 @@ type StreamingNodeManagerCollectStatusRequest struct { func (x *StreamingNodeManagerCollectStatusRequest) Reset() { *x = StreamingNodeManagerCollectStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[55] + mi := &file_streaming_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3591,7 +3259,7 @@ func (x *StreamingNodeManagerCollectStatusRequest) String() string { func (*StreamingNodeManagerCollectStatusRequest) ProtoMessage() {} func (x *StreamingNodeManagerCollectStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[55] + mi := &file_streaming_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3604,7 +3272,7 @@ func (x *StreamingNodeManagerCollectStatusRequest) ProtoReflect() protoreflect.M // Deprecated: Use StreamingNodeManagerCollectStatusRequest.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerCollectStatusRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{55} + return file_streaming_proto_rawDescGZIP(), []int{49} } type StreamingNodeBalanceAttributes struct { @@ -3616,7 +3284,7 @@ type StreamingNodeBalanceAttributes struct { func (x *StreamingNodeBalanceAttributes) Reset() { *x = StreamingNodeBalanceAttributes{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[56] + mi := &file_streaming_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3629,7 +3297,7 @@ func (x *StreamingNodeBalanceAttributes) String() string { func (*StreamingNodeBalanceAttributes) ProtoMessage() {} func (x *StreamingNodeBalanceAttributes) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[56] + mi := &file_streaming_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3642,7 +3310,7 @@ func (x *StreamingNodeBalanceAttributes) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingNodeBalanceAttributes.ProtoReflect.Descriptor instead. func (*StreamingNodeBalanceAttributes) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{56} + return file_streaming_proto_rawDescGZIP(), []int{50} } type StreamingNodeManagerCollectStatusResponse struct { @@ -3656,7 +3324,7 @@ type StreamingNodeManagerCollectStatusResponse struct { func (x *StreamingNodeManagerCollectStatusResponse) Reset() { *x = StreamingNodeManagerCollectStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[57] + mi := &file_streaming_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3669,7 +3337,7 @@ func (x *StreamingNodeManagerCollectStatusResponse) String() string { func (*StreamingNodeManagerCollectStatusResponse) ProtoMessage() {} func (x *StreamingNodeManagerCollectStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[57] + mi := &file_streaming_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3682,7 +3350,7 @@ func (x *StreamingNodeManagerCollectStatusResponse) ProtoReflect() protoreflect. // Deprecated: Use StreamingNodeManagerCollectStatusResponse.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerCollectStatusResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{57} + return file_streaming_proto_rawDescGZIP(), []int{51} } func (x *StreamingNodeManagerCollectStatusResponse) GetBalanceAttributes() *StreamingNodeBalanceAttributes { @@ -3716,7 +3384,7 @@ type SegmentAssignmentMeta struct { func (x *SegmentAssignmentMeta) Reset() { *x = SegmentAssignmentMeta{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[58] + mi := &file_streaming_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3729,7 +3397,7 @@ func (x *SegmentAssignmentMeta) String() string { func (*SegmentAssignmentMeta) ProtoMessage() {} func (x *SegmentAssignmentMeta) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[58] + mi := &file_streaming_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3742,7 +3410,7 @@ func (x *SegmentAssignmentMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentAssignmentMeta.ProtoReflect.Descriptor instead. func (*SegmentAssignmentMeta) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{58} + return file_streaming_proto_rawDescGZIP(), []int{52} } func (x *SegmentAssignmentMeta) GetCollectionId() int64 { @@ -3805,7 +3473,7 @@ type SegmentAssignmentStat struct { func (x *SegmentAssignmentStat) Reset() { *x = SegmentAssignmentStat{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[59] + mi := &file_streaming_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3818,7 +3486,7 @@ func (x *SegmentAssignmentStat) String() string { func (*SegmentAssignmentStat) ProtoMessage() {} func (x *SegmentAssignmentStat) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[59] + mi := &file_streaming_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3831,7 +3499,7 @@ func (x *SegmentAssignmentStat) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentAssignmentStat.ProtoReflect.Descriptor instead. func (*SegmentAssignmentStat) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{59} + return file_streaming_proto_rawDescGZIP(), []int{53} } func (x *SegmentAssignmentStat) GetMaxBinarySize() uint64 { @@ -3895,7 +3563,7 @@ type WALCheckpoint struct { func (x *WALCheckpoint) Reset() { *x = WALCheckpoint{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[60] + mi := &file_streaming_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3908,7 +3576,7 @@ func (x *WALCheckpoint) String() string { func (*WALCheckpoint) ProtoMessage() {} func (x *WALCheckpoint) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[60] + mi := &file_streaming_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3921,7 +3589,7 @@ func (x *WALCheckpoint) ProtoReflect() protoreflect.Message { // Deprecated: Use WALCheckpoint.ProtoReflect.Descriptor instead. func (*WALCheckpoint) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{60} + return file_streaming_proto_rawDescGZIP(), []int{54} } func (x *WALCheckpoint) GetMessageID() *messagespb.MessageID { @@ -4013,614 +3681,565 @@ var file_streaming_proto_rawDesc = []byte{ 0x61, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x16, 0x0a, 0x14, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, - 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd8, 0x01, 0x0a, 0x15, 0x42, 0x72, - 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x68, 0x0a, 0x12, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x5f, 0x77, 0x61, 0x74, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x38, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, - 0x73, 0x74, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x74, - 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x10, 0x63, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x12, 0x4a, 0x0a, - 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x72, 0x6f, 0x61, 0x64, - 0x63, 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x22, 0x5f, 0x0a, 0x20, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, - 0x74, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x74, 0x63, - 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3b, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x1c, 0x0a, 0x1a, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x72, - 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0xc7, 0x01, 0x0a, 0x16, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, - 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, - 0x0a, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x64, 0x6f, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, - 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, - 0x44, 0x6f, 0x6e, 0x65, 0x12, 0x4b, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, - 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x5a, 0x0a, - 0x1b, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x57, - 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3b, 0x0a, 0x05, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 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, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x1d, 0x0a, 0x1b, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd1, 0x01, 0x0a, 0x19, 0x41, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x59, 0x0a, 0x0c, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, - 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x41, 0x73, 0x73, 0x69, - 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x4e, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, - 0x65, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x22, 0x9a, 0x01, 0x0a, - 0x1c, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, - 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, - 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, - 0x38, 0x0a, 0x03, 0x65, 0x72, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x52, 0x03, 0x65, 0x72, 0x72, 0x22, 0x20, 0x0a, 0x1e, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, - 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xe4, 0x01, 0x0a, 0x1a, + 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd1, 0x01, 0x0a, 0x19, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x59, 0x0a, 0x0c, 0x72, 0x65, 0x70, 0x6f, 0x72, + 0x74, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x12, 0x4e, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x0f, 0x66, 0x75, - 0x6c, 0x6c, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x46, 0x75, 0x6c, - 0x6c, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, - 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x57, 0x69, 0x74, 0x68, 0x56, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0e, 0x66, 0x75, 0x6c, 0x6c, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x4f, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, - 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, - 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, - 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0xba, 0x01, 0x0a, 0x26, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x57, 0x69, 0x74, 0x68, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, - 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x50, - 0x61, 0x69, 0x72, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x51, 0x0a, 0x0b, - 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x52, 0x0b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, - 0x21, 0x0a, 0x1f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, - 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x4a, 0x0a, 0x11, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, - 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, - 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, 0x9a, - 0x01, 0x0a, 0x17, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x3d, 0x0a, 0x04, 0x6e, 0x6f, - 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, - 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x40, 0x0a, 0x08, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x0d, - 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x2a, 0x0a, - 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x30, 0x0a, 0x06, 0x6c, 0x61, 0x74, - 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, - 0x79, 0x48, 0x00, 0x52, 0x06, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x12, 0x41, 0x0a, 0x0a, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, - 0x44, 0x48, 0x00, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x12, 0x43, - 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x49, 0x44, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x41, 0x66, - 0x74, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x9d, 0x02, - 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, - 0x53, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x5f, 0x67, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, - 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, - 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, - 0x63, 0x6b, 0x47, 0x74, 0x12, 0x56, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, - 0x6b, 0x5f, 0x67, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, - 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x45, 0x48, 0x00, 0x52, - 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x74, 0x65, 0x12, 0x55, 0x0a, 0x0c, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, - 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, - 0x79, 0x70, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, 0x36, 0x0a, - 0x17, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, - 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, - 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, - 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, 0x37, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, - 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, - 0x45, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, 0x63, - 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x0d, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0e, 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, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, - 0x70, 0x65, 0x73, 0x22, 0x61, 0x0a, 0x0e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, - 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x39, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x22, 0xac, 0x01, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x49, 0x0a, 0x07, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x12, 0x44, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, - 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, + 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x22, 0x9a, 0x01, + 0x0a, 0x1c, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x22, 0x70, 0x0a, 0x15, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x0f, 0x50, - 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, - 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, - 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, - 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x4a, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x12, 0x38, 0x0a, 0x03, 0x65, 0x72, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x03, 0x65, 0x72, 0x72, 0x22, 0x20, 0x0a, 0x1e, 0x43, 0x6c, + 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, + 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xe4, 0x01, 0x0a, + 0x1a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, + 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x0f, 0x66, + 0x75, 0x6c, 0x6c, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x46, 0x75, + 0x6c, 0x6c, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x57, 0x69, 0x74, 0x68, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0e, 0x66, 0x75, 0x6c, 0x6c, 0x41, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x4f, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, + 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, + 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0xba, 0x01, 0x0a, 0x26, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x57, 0x69, 0x74, 0x68, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3d, + 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x51, 0x0a, + 0x0b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x22, 0x21, 0x0a, 0x1f, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x4a, 0x0a, 0x11, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, + 0x9a, 0x01, 0x0a, 0x17, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, + 0x65, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x3d, 0x0a, 0x04, 0x6e, + 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x12, 0x45, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x61, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x19, 0x0a, 0x08, 0x77, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x12, - 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, - 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x22, 0xd3, 0x01, 0x0a, 0x16, 0x50, - 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, - 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x06, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3e, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x05, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0xdb, 0x01, 0x0a, 0x1c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x12, 0x30, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, - 0x02, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x74, 0x69, 0x63, 0x6b, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x74, 0x69, 0x63, 0x6b, 0x12, - 0x41, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x54, 0x78, 0x6e, 0x43, - 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0a, 0x74, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, - 0x78, 0x74, 0x12, 0x2a, 0x0a, 0x05, 0x65, 0x78, 0x74, 0x72, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x05, 0x65, 0x78, 0x74, 0x72, 0x61, 0x22, 0x17, - 0x0a, 0x15, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x03, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x71, 0x0a, 0x18, 0x63, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, + 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x40, 0x0a, 0x08, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x16, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x74, 0x0a, - 0x19, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x17, 0x63, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x73, 0x12, 0x5d, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x76, 0x63, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x08, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0xff, 0x01, 0x0a, + 0x0d, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x2a, + 0x0a, 0x03, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6c, 0x6c, 0x12, 0x30, 0x0a, 0x06, 0x6c, 0x61, + 0x74, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x48, 0x00, 0x52, 0x06, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x12, 0x41, 0x0a, 0x0a, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x49, 0x44, 0x48, 0x00, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x12, + 0x43, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x41, + 0x66, 0x74, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x9d, + 0x02, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x12, 0x53, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x5f, 0x67, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, + 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x54, + 0x69, 0x63, 0x6b, 0x47, 0x74, 0x12, 0x56, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, + 0x63, 0x6b, 0x5f, 0x67, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x45, 0x48, 0x00, + 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x74, 0x65, 0x12, 0x55, 0x0a, + 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, + 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x54, 0x79, 0x70, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, 0x36, + 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, + 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, 0x54, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, + 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, 0x37, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, + 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x47, + 0x54, 0x45, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, + 0x63, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x0d, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0e, 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, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, + 0x79, 0x70, 0x65, 0x73, 0x22, 0x61, 0x0a, 0x0e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x39, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x63, 0x61, 0x75, 0x73, 0x65, 0x22, 0xac, 0x01, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x49, 0x0a, 0x07, 0x70, 0x72, + 0x6f, 0x64, 0x75, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, + 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x44, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, + 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x22, 0x70, 0x0a, 0x15, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xfa, 0x01, 0x0a, 0x0f, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x48, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, + 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, + 0x00, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x4a, 0x0a, 0x07, 0x70, 0x72, 0x6f, + 0x64, 0x75, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, + 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x45, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, + 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, + 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x61, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x77, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, + 0x12, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x70, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x22, 0xd3, 0x01, 0x0a, 0x16, + 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, + 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x06, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3e, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0xdb, 0x01, 0x0a, 0x1c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x12, 0x30, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, + 0x52, 0x02, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x74, 0x69, 0x63, 0x6b, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x74, 0x69, 0x6d, 0x65, 0x74, 0x69, 0x63, 0x6b, + 0x12, 0x41, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x54, 0x78, 0x6e, + 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0a, 0x74, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, + 0x65, 0x78, 0x74, 0x12, 0x2a, 0x0a, 0x05, 0x65, 0x78, 0x74, 0x72, 0x61, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x05, 0x65, 0x78, 0x74, 0x72, 0x61, 0x22, + 0x17, 0x0a, 0x15, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x03, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x71, 0x0a, 0x18, 0x63, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x16, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x74, + 0x0a, 0x19, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x17, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x73, 0x12, 0x5d, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x76, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x12, 0x44, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, + 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x15, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x82, 0x01, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x60, 0x0a, 0x10, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0f, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0xd9, 0x01, 0x0a, + 0x1d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, + 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x4c, 0x0a, 0x0e, 0x64, 0x65, + 0x6c, 0x69, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, + 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0d, 0x64, 0x65, 0x6c, 0x69, 0x76, + 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x69, + 0x76, 0x65, 0x72, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, + 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0e, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, + 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x1f, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x61, 0x0a, 0x10, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0f, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, + 0x8f, 0x01, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3e, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x3f, 0x0a, 0x1c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x12, 0x44, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, - 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, - 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x59, 0x0a, 0x15, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x49, 0x64, 0x22, 0x40, 0x0a, 0x1d, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x49, 0x64, 0x22, 0xa2, 0x04, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x12, 0x49, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x48, 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x61, 0x0a, + 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, + 0x52, 0x0e, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x12, 0x64, 0x0a, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x5e, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, + 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x45, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, + 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, + 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x61, 0x0a, 0x16, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x77, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, + 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x22, 0x7b, 0x0a, 0x15, + 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x41, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, + 0x49, 0x6d, 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x43, 0x6c, 0x6f, + 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x65, 0x0a, 0x21, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, + 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x24, 0x0a, 0x22, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x65, 0x0a, 0x21, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, + 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x82, 0x01, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x60, 0x0a, 0x10, 0x63, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0f, 0x63, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0xd9, 0x01, 0x0a, 0x1d, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, - 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x4c, 0x0a, 0x0e, 0x64, 0x65, 0x6c, - 0x69, 0x76, 0x65, 0x72, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, - 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0d, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, - 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x4e, 0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x69, 0x76, - 0x65, 0x72, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x44, 0x65, 0x6c, 0x69, 0x76, 0x65, - 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x0e, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, - 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x1f, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x61, 0x0a, 0x10, 0x63, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0f, 0x63, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x22, 0x8f, - 0x01, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x49, 0x64, 0x12, 0x3e, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x05, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x3f, 0x0a, 0x1c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, - 0x64, 0x22, 0x40, 0x0a, 0x1d, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x49, 0x64, 0x22, 0xa2, 0x04, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x12, 0x49, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x48, 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x61, 0x0a, 0x0f, - 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, - 0x0e, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, - 0x64, 0x0a, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x5e, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x76, - 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x56, 0x63, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x45, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6c, - 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, - 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x61, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x77, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x77, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, - 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x22, 0x7b, 0x0a, 0x15, 0x43, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x41, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x49, - 0x6d, 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x43, 0x6c, 0x6f, 0x73, - 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x65, 0x0a, 0x21, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, - 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, - 0x67, 0x2e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, - 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x24, 0x0a, 0x22, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x65, - 0x0a, 0x21, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x70, 0x63, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x24, 0x0a, 0x22, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, - 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2a, 0x0a, 0x28, 0x53, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x22, 0x24, 0x0a, 0x22, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, + 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2a, 0x0a, 0x28, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x20, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0x92, 0x01, 0x0a, 0x29, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x20, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x41, - 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0x92, 0x01, 0x0a, 0x29, 0x53, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x65, 0x0a, 0x12, 0x62, 0x61, 0x6c, 0x61, 0x6e, - 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x11, 0x62, 0x61, 0x6c, - 0x61, 0x6e, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0xa3, - 0x02, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, - 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, - 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, - 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x44, 0x0a, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x41, 0x0a, 0x04, 0x73, 0x74, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x52, 0x04, - 0x73, 0x74, 0x61, 0x74, 0x22, 0x87, 0x03, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x12, 0x26, - 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x42, 0x69, 0x6e, 0x61, - 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, - 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x69, - 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x69, - 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x69, 0x6e, 0x73, 0x65, 0x72, - 0x74, 0x65, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x40, 0x0a, - 0x1c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x1a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, - 0x4d, 0x0a, 0x23, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, - 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, - 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x6c, 0x61, - 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x25, - 0x0a, 0x0e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x43, 0x6f, - 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x18, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, - 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, 0x4f, - 0x0a, 0x0d, 0x57, 0x41, 0x4c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, - 0x3e, 0x0a, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x49, 0x44, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x2a, - 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74, 0x61, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, - 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, - 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, - 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, - 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, 0x0a, - 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, - 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, - 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, - 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x45, 0x44, - 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, - 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x41, 0x56, 0x41, 0x49, - 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x9a, 0x01, 0x0a, 0x12, 0x42, 0x72, 0x6f, 0x61, - 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x20, - 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, - 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, - 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, - 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, - 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, - 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x4f, 0x4e, 0x45, 0x10, - 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, - 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x41, 0x49, 0x54, 0x5f, 0x41, - 0x43, 0x4b, 0x10, 0x03, 0x2a, 0x82, 0x04, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24, 0x0a, - 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, - 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, 0x53, - 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, - 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46, 0x45, - 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55, 0x54, - 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, - 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12, 0x29, - 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, - 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, - 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, - 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, 0x4f, - 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, - 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, - 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54, 0x52, - 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, - 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12, 0x26, - 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, - 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58, 0x50, - 0x49, 0x52, 0x45, 0x44, 0x10, 0x09, 0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, - 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, - 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, - 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, - 0x45, 0x5f, 0x41, 0x43, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x0c, 0x12, 0x1b, 0x0a, 0x16, - 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, - 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, - 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, - 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, - 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, - 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, - 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, - 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52, 0x4f, - 0x57, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, - 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x45, 0x5f, 0x53, 0x45, 0x41, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, 0x53, - 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, - 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44, 0x10, - 0x04, 0x32, 0x89, 0x01, 0x0a, 0x19, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, - 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, - 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, - 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, - 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x32, 0xd6, 0x02, - 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, - 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x12, 0x62, 0x0a, 0x09, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, - 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x03, 0x41, 0x63, 0x6b, 0x12, 0x2b, 0x2e, 0x6d, 0x69, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x65, 0x0a, 0x12, 0x62, 0x61, 0x6c, 0x61, + 0x6e, 0x63, 0x65, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x11, 0x62, 0x61, + 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, + 0xa3, 0x02, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, + 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, + 0x12, 0x1a, 0x0a, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x44, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, - 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x41, 0x0a, 0x04, 0x73, 0x74, 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x52, + 0x04, 0x73, 0x74, 0x61, 0x74, 0x22, 0x87, 0x03, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x12, + 0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x42, 0x69, 0x6e, + 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, + 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x30, 0x0a, 0x14, + 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x69, 0x6e, 0x73, 0x65, + 0x72, 0x74, 0x65, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x40, + 0x0a, 0x1c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x1a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, + 0x12, 0x4d, 0x0a, 0x23, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, + 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x6c, + 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, + 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x18, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, + 0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, + 0x4f, 0x0a, 0x0d, 0x57, 0x41, 0x4c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x12, 0x3e, 0x0a, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, + 0x2a, 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74, + 0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, + 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, + 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x43, 0x48, 0x41, 0x4e, + 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, + 0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, + 0x0a, 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, + 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x49, 0x4e, 0x47, 0x10, + 0x02, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, + 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x45, + 0x44, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, + 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x41, 0x56, 0x41, + 0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x9a, 0x01, 0x0a, 0x12, 0x42, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, + 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, + 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, + 0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, + 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x4f, 0x4e, 0x45, + 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x41, 0x49, 0x54, 0x5f, + 0x41, 0x43, 0x4b, 0x10, 0x03, 0x2a, 0x82, 0x04, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24, + 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, + 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, + 0x53, 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, + 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46, + 0x45, 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55, + 0x54, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, + 0x44, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12, + 0x29, 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e, + 0x4e, 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, + 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, + 0x4f, 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, + 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54, + 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, + 0x41, 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12, + 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58, + 0x50, 0x49, 0x52, 0x45, 0x44, 0x10, 0x09, 0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, + 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, + 0x41, 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, + 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, + 0x52, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, + 0x43, 0x45, 0x5f, 0x41, 0x43, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x0c, 0x12, 0x1b, 0x0a, + 0x16, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, + 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, + 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, + 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, + 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, + 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, + 0x01, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, + 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52, + 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45, + 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, + 0x41, 0x54, 0x45, 0x5f, 0x53, 0x45, 0x41, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, + 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, + 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44, + 0x10, 0x04, 0x32, 0x89, 0x01, 0x0a, 0x19, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, + 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x32, 0xe8, + 0x01, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, + 0x64, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 0x62, 0x0a, 0x09, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, - 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, 0x6b, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x05, 0x57, 0x61, 0x74, 0x63, - 0x68, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, - 0x63, 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, - 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xa5, 0x01, 0x0a, 0x1f, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, - 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x12, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, - 0x72, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x03, 0x41, 0x63, 0x6b, 0x12, 0x2b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, + 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, 0x6b, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa5, 0x01, 0x0a, 0x1f, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x41, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, + 0x0a, 0x12, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, + 0x6f, 0x76, 0x65, 0x72, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xe1, - 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, - 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x60, - 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, - 0x12, 0x60, 0x0a, 0x07, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, - 0x30, 0x01, 0x32, 0xbe, 0x03, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, - 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x12, 0x39, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, - 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, - 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, - 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, + 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, + 0x01, 0x32, 0xe1, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, + 0x6f, 0x64, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 0x60, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, - 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x0d, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, - 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, + 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, + 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x07, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, - 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x42, 0x36, 0x5a, 0x34, 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, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x33, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbe, 0x03, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, + 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x52, 0x65, + 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x96, 0x01, + 0x0a, 0x0d, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, + 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x41, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x36, 0x5a, 0x34, 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, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, 0x62, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -4636,7 +4255,7 @@ func file_streaming_proto_rawDescGZIP() []byte { } var file_streaming_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 62) +var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 56) var file_streaming_proto_goTypes = []interface{}{ (PChannelMetaState)(0), // 0: milvus.proto.streaming.PChannelMetaState (BroadcastTaskState)(0), // 1: milvus.proto.streaming.BroadcastTaskState @@ -4651,166 +4270,151 @@ var file_streaming_proto_goTypes = []interface{}{ (*BroadcastResponse)(nil), // 10: milvus.proto.streaming.BroadcastResponse (*BroadcastAckRequest)(nil), // 11: milvus.proto.streaming.BroadcastAckRequest (*BroadcastAckResponse)(nil), // 12: milvus.proto.streaming.BroadcastAckResponse - (*BroadcastWatchRequest)(nil), // 13: milvus.proto.streaming.BroadcastWatchRequest - (*BroadcastCreateEventWatchRequest)(nil), // 14: milvus.proto.streaming.BroadcastCreateEventWatchRequest - (*CloseBroadcastWatchRequest)(nil), // 15: milvus.proto.streaming.CloseBroadcastWatchRequest - (*BroadcastWatchResponse)(nil), // 16: milvus.proto.streaming.BroadcastWatchResponse - (*BroadcastEventWatchResponse)(nil), // 17: milvus.proto.streaming.BroadcastEventWatchResponse - (*CloseBroadcastWatchResponse)(nil), // 18: milvus.proto.streaming.CloseBroadcastWatchResponse - (*AssignmentDiscoverRequest)(nil), // 19: milvus.proto.streaming.AssignmentDiscoverRequest - (*ReportAssignmentErrorRequest)(nil), // 20: milvus.proto.streaming.ReportAssignmentErrorRequest - (*CloseAssignmentDiscoverRequest)(nil), // 21: milvus.proto.streaming.CloseAssignmentDiscoverRequest - (*AssignmentDiscoverResponse)(nil), // 22: milvus.proto.streaming.AssignmentDiscoverResponse - (*FullStreamingNodeAssignmentWithVersion)(nil), // 23: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion - (*CloseAssignmentDiscoverResponse)(nil), // 24: milvus.proto.streaming.CloseAssignmentDiscoverResponse - (*StreamingNodeInfo)(nil), // 25: milvus.proto.streaming.StreamingNodeInfo - (*StreamingNodeAssignment)(nil), // 26: milvus.proto.streaming.StreamingNodeAssignment - (*DeliverPolicy)(nil), // 27: milvus.proto.streaming.DeliverPolicy - (*DeliverFilter)(nil), // 28: milvus.proto.streaming.DeliverFilter - (*DeliverFilterTimeTickGT)(nil), // 29: milvus.proto.streaming.DeliverFilterTimeTickGT - (*DeliverFilterTimeTickGTE)(nil), // 30: milvus.proto.streaming.DeliverFilterTimeTickGTE - (*DeliverFilterMessageType)(nil), // 31: milvus.proto.streaming.DeliverFilterMessageType - (*StreamingError)(nil), // 32: milvus.proto.streaming.StreamingError - (*ProduceRequest)(nil), // 33: milvus.proto.streaming.ProduceRequest - (*CreateProducerRequest)(nil), // 34: milvus.proto.streaming.CreateProducerRequest - (*ProduceMessageRequest)(nil), // 35: milvus.proto.streaming.ProduceMessageRequest - (*CloseProducerRequest)(nil), // 36: milvus.proto.streaming.CloseProducerRequest - (*ProduceResponse)(nil), // 37: milvus.proto.streaming.ProduceResponse - (*CreateProducerResponse)(nil), // 38: milvus.proto.streaming.CreateProducerResponse - (*ProduceMessageResponse)(nil), // 39: milvus.proto.streaming.ProduceMessageResponse - (*ProduceMessageResponseResult)(nil), // 40: milvus.proto.streaming.ProduceMessageResponseResult - (*CloseProducerResponse)(nil), // 41: milvus.proto.streaming.CloseProducerResponse - (*ConsumeRequest)(nil), // 42: milvus.proto.streaming.ConsumeRequest - (*CloseConsumerRequest)(nil), // 43: milvus.proto.streaming.CloseConsumerRequest - (*CreateConsumerRequest)(nil), // 44: milvus.proto.streaming.CreateConsumerRequest - (*CreateVChannelConsumersRequest)(nil), // 45: milvus.proto.streaming.CreateVChannelConsumersRequest - (*CreateVChannelConsumerRequest)(nil), // 46: milvus.proto.streaming.CreateVChannelConsumerRequest - (*CreateVChannelConsumersResponse)(nil), // 47: milvus.proto.streaming.CreateVChannelConsumersResponse - (*CreateVChannelConsumerResponse)(nil), // 48: milvus.proto.streaming.CreateVChannelConsumerResponse - (*CloseVChannelConsumerRequest)(nil), // 49: milvus.proto.streaming.CloseVChannelConsumerRequest - (*CloseVChannelConsumerResponse)(nil), // 50: milvus.proto.streaming.CloseVChannelConsumerResponse - (*ConsumeResponse)(nil), // 51: milvus.proto.streaming.ConsumeResponse - (*CreateConsumerResponse)(nil), // 52: milvus.proto.streaming.CreateConsumerResponse - (*ConsumeMessageReponse)(nil), // 53: milvus.proto.streaming.ConsumeMessageReponse - (*CloseConsumerResponse)(nil), // 54: milvus.proto.streaming.CloseConsumerResponse - (*StreamingNodeManagerAssignRequest)(nil), // 55: milvus.proto.streaming.StreamingNodeManagerAssignRequest - (*StreamingNodeManagerAssignResponse)(nil), // 56: milvus.proto.streaming.StreamingNodeManagerAssignResponse - (*StreamingNodeManagerRemoveRequest)(nil), // 57: milvus.proto.streaming.StreamingNodeManagerRemoveRequest - (*StreamingNodeManagerRemoveResponse)(nil), // 58: milvus.proto.streaming.StreamingNodeManagerRemoveResponse - (*StreamingNodeManagerCollectStatusRequest)(nil), // 59: milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest - (*StreamingNodeBalanceAttributes)(nil), // 60: milvus.proto.streaming.StreamingNodeBalanceAttributes - (*StreamingNodeManagerCollectStatusResponse)(nil), // 61: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse - (*SegmentAssignmentMeta)(nil), // 62: milvus.proto.streaming.SegmentAssignmentMeta - (*SegmentAssignmentStat)(nil), // 63: milvus.proto.streaming.SegmentAssignmentStat - (*WALCheckpoint)(nil), // 64: milvus.proto.streaming.WALCheckpoint - nil, // 65: milvus.proto.streaming.BroadcastResponse.ResultsEntry - (*messagespb.Message)(nil), // 66: milvus.proto.messages.Message - (*messagespb.BroadcastEvent)(nil), // 67: milvus.proto.messages.BroadcastEvent - (*emptypb.Empty)(nil), // 68: google.protobuf.Empty - (*messagespb.MessageID)(nil), // 69: milvus.proto.messages.MessageID - (messagespb.MessageType)(0), // 70: milvus.proto.messages.MessageType - (*messagespb.TxnContext)(nil), // 71: milvus.proto.messages.TxnContext - (*anypb.Any)(nil), // 72: google.protobuf.Any - (*messagespb.ImmutableMessage)(nil), // 73: milvus.proto.messages.ImmutableMessage - (*milvuspb.GetComponentStatesRequest)(nil), // 74: milvus.proto.milvus.GetComponentStatesRequest - (*milvuspb.ComponentStates)(nil), // 75: milvus.proto.milvus.ComponentStates + (*AssignmentDiscoverRequest)(nil), // 13: milvus.proto.streaming.AssignmentDiscoverRequest + (*ReportAssignmentErrorRequest)(nil), // 14: milvus.proto.streaming.ReportAssignmentErrorRequest + (*CloseAssignmentDiscoverRequest)(nil), // 15: milvus.proto.streaming.CloseAssignmentDiscoverRequest + (*AssignmentDiscoverResponse)(nil), // 16: milvus.proto.streaming.AssignmentDiscoverResponse + (*FullStreamingNodeAssignmentWithVersion)(nil), // 17: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion + (*CloseAssignmentDiscoverResponse)(nil), // 18: milvus.proto.streaming.CloseAssignmentDiscoverResponse + (*StreamingNodeInfo)(nil), // 19: milvus.proto.streaming.StreamingNodeInfo + (*StreamingNodeAssignment)(nil), // 20: milvus.proto.streaming.StreamingNodeAssignment + (*DeliverPolicy)(nil), // 21: milvus.proto.streaming.DeliverPolicy + (*DeliverFilter)(nil), // 22: milvus.proto.streaming.DeliverFilter + (*DeliverFilterTimeTickGT)(nil), // 23: milvus.proto.streaming.DeliverFilterTimeTickGT + (*DeliverFilterTimeTickGTE)(nil), // 24: milvus.proto.streaming.DeliverFilterTimeTickGTE + (*DeliverFilterMessageType)(nil), // 25: milvus.proto.streaming.DeliverFilterMessageType + (*StreamingError)(nil), // 26: milvus.proto.streaming.StreamingError + (*ProduceRequest)(nil), // 27: milvus.proto.streaming.ProduceRequest + (*CreateProducerRequest)(nil), // 28: milvus.proto.streaming.CreateProducerRequest + (*ProduceMessageRequest)(nil), // 29: milvus.proto.streaming.ProduceMessageRequest + (*CloseProducerRequest)(nil), // 30: milvus.proto.streaming.CloseProducerRequest + (*ProduceResponse)(nil), // 31: milvus.proto.streaming.ProduceResponse + (*CreateProducerResponse)(nil), // 32: milvus.proto.streaming.CreateProducerResponse + (*ProduceMessageResponse)(nil), // 33: milvus.proto.streaming.ProduceMessageResponse + (*ProduceMessageResponseResult)(nil), // 34: milvus.proto.streaming.ProduceMessageResponseResult + (*CloseProducerResponse)(nil), // 35: milvus.proto.streaming.CloseProducerResponse + (*ConsumeRequest)(nil), // 36: milvus.proto.streaming.ConsumeRequest + (*CloseConsumerRequest)(nil), // 37: milvus.proto.streaming.CloseConsumerRequest + (*CreateConsumerRequest)(nil), // 38: milvus.proto.streaming.CreateConsumerRequest + (*CreateVChannelConsumersRequest)(nil), // 39: milvus.proto.streaming.CreateVChannelConsumersRequest + (*CreateVChannelConsumerRequest)(nil), // 40: milvus.proto.streaming.CreateVChannelConsumerRequest + (*CreateVChannelConsumersResponse)(nil), // 41: milvus.proto.streaming.CreateVChannelConsumersResponse + (*CreateVChannelConsumerResponse)(nil), // 42: milvus.proto.streaming.CreateVChannelConsumerResponse + (*CloseVChannelConsumerRequest)(nil), // 43: milvus.proto.streaming.CloseVChannelConsumerRequest + (*CloseVChannelConsumerResponse)(nil), // 44: milvus.proto.streaming.CloseVChannelConsumerResponse + (*ConsumeResponse)(nil), // 45: milvus.proto.streaming.ConsumeResponse + (*CreateConsumerResponse)(nil), // 46: milvus.proto.streaming.CreateConsumerResponse + (*ConsumeMessageReponse)(nil), // 47: milvus.proto.streaming.ConsumeMessageReponse + (*CloseConsumerResponse)(nil), // 48: milvus.proto.streaming.CloseConsumerResponse + (*StreamingNodeManagerAssignRequest)(nil), // 49: milvus.proto.streaming.StreamingNodeManagerAssignRequest + (*StreamingNodeManagerAssignResponse)(nil), // 50: milvus.proto.streaming.StreamingNodeManagerAssignResponse + (*StreamingNodeManagerRemoveRequest)(nil), // 51: milvus.proto.streaming.StreamingNodeManagerRemoveRequest + (*StreamingNodeManagerRemoveResponse)(nil), // 52: milvus.proto.streaming.StreamingNodeManagerRemoveResponse + (*StreamingNodeManagerCollectStatusRequest)(nil), // 53: milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + (*StreamingNodeBalanceAttributes)(nil), // 54: milvus.proto.streaming.StreamingNodeBalanceAttributes + (*StreamingNodeManagerCollectStatusResponse)(nil), // 55: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + (*SegmentAssignmentMeta)(nil), // 56: milvus.proto.streaming.SegmentAssignmentMeta + (*SegmentAssignmentStat)(nil), // 57: milvus.proto.streaming.SegmentAssignmentStat + (*WALCheckpoint)(nil), // 58: milvus.proto.streaming.WALCheckpoint + nil, // 59: milvus.proto.streaming.BroadcastResponse.ResultsEntry + (*messagespb.Message)(nil), // 60: milvus.proto.messages.Message + (*emptypb.Empty)(nil), // 61: google.protobuf.Empty + (*messagespb.MessageID)(nil), // 62: milvus.proto.messages.MessageID + (messagespb.MessageType)(0), // 63: milvus.proto.messages.MessageType + (*messagespb.TxnContext)(nil), // 64: milvus.proto.messages.TxnContext + (*anypb.Any)(nil), // 65: google.protobuf.Any + (*messagespb.ImmutableMessage)(nil), // 66: milvus.proto.messages.ImmutableMessage + (*milvuspb.GetComponentStatesRequest)(nil), // 67: milvus.proto.milvus.GetComponentStatesRequest + (*milvuspb.ComponentStates)(nil), // 68: milvus.proto.milvus.ComponentStates } var file_streaming_proto_depIdxs = []int32{ - 25, // 0: milvus.proto.streaming.PChannelAssignmentLog.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 19, // 0: milvus.proto.streaming.PChannelAssignmentLog.node:type_name -> milvus.proto.streaming.StreamingNodeInfo 4, // 1: milvus.proto.streaming.PChannelMeta.channel:type_name -> milvus.proto.streaming.PChannelInfo - 25, // 2: milvus.proto.streaming.PChannelMeta.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 19, // 2: milvus.proto.streaming.PChannelMeta.node:type_name -> milvus.proto.streaming.StreamingNodeInfo 0, // 3: milvus.proto.streaming.PChannelMeta.state:type_name -> milvus.proto.streaming.PChannelMetaState 5, // 4: milvus.proto.streaming.PChannelMeta.histories:type_name -> milvus.proto.streaming.PChannelAssignmentLog - 66, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message + 60, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message 1, // 6: milvus.proto.streaming.BroadcastTask.state:type_name -> milvus.proto.streaming.BroadcastTaskState - 66, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message - 65, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry - 14, // 9: milvus.proto.streaming.BroadcastWatchRequest.create_event_watch:type_name -> milvus.proto.streaming.BroadcastCreateEventWatchRequest - 15, // 10: milvus.proto.streaming.BroadcastWatchRequest.close:type_name -> milvus.proto.streaming.CloseBroadcastWatchRequest - 67, // 11: milvus.proto.streaming.BroadcastCreateEventWatchRequest.event:type_name -> milvus.proto.messages.BroadcastEvent - 17, // 12: milvus.proto.streaming.BroadcastWatchResponse.event_done:type_name -> milvus.proto.streaming.BroadcastEventWatchResponse - 18, // 13: milvus.proto.streaming.BroadcastWatchResponse.close:type_name -> milvus.proto.streaming.CloseBroadcastWatchResponse - 67, // 14: milvus.proto.streaming.BroadcastEventWatchResponse.event:type_name -> milvus.proto.messages.BroadcastEvent - 20, // 15: milvus.proto.streaming.AssignmentDiscoverRequest.report_error:type_name -> milvus.proto.streaming.ReportAssignmentErrorRequest - 21, // 16: milvus.proto.streaming.AssignmentDiscoverRequest.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverRequest - 4, // 17: milvus.proto.streaming.ReportAssignmentErrorRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 32, // 18: milvus.proto.streaming.ReportAssignmentErrorRequest.err:type_name -> milvus.proto.streaming.StreamingError - 23, // 19: milvus.proto.streaming.AssignmentDiscoverResponse.full_assignment:type_name -> milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion - 24, // 20: milvus.proto.streaming.AssignmentDiscoverResponse.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverResponse - 7, // 21: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.version:type_name -> milvus.proto.streaming.VersionPair - 26, // 22: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.assignments:type_name -> milvus.proto.streaming.StreamingNodeAssignment - 25, // 23: milvus.proto.streaming.StreamingNodeAssignment.node:type_name -> milvus.proto.streaming.StreamingNodeInfo - 4, // 24: milvus.proto.streaming.StreamingNodeAssignment.channels:type_name -> milvus.proto.streaming.PChannelInfo - 68, // 25: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty - 68, // 26: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty - 69, // 27: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID - 69, // 28: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID - 29, // 29: milvus.proto.streaming.DeliverFilter.time_tick_gt:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGT - 30, // 30: milvus.proto.streaming.DeliverFilter.time_tick_gte:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGTE - 31, // 31: milvus.proto.streaming.DeliverFilter.message_type:type_name -> milvus.proto.streaming.DeliverFilterMessageType - 70, // 32: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType - 2, // 33: milvus.proto.streaming.StreamingError.code:type_name -> milvus.proto.streaming.StreamingCode - 35, // 34: milvus.proto.streaming.ProduceRequest.produce:type_name -> milvus.proto.streaming.ProduceMessageRequest - 36, // 35: milvus.proto.streaming.ProduceRequest.close:type_name -> milvus.proto.streaming.CloseProducerRequest - 4, // 36: milvus.proto.streaming.CreateProducerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 66, // 37: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message - 38, // 38: milvus.proto.streaming.ProduceResponse.create:type_name -> milvus.proto.streaming.CreateProducerResponse - 39, // 39: milvus.proto.streaming.ProduceResponse.produce:type_name -> milvus.proto.streaming.ProduceMessageResponse - 41, // 40: milvus.proto.streaming.ProduceResponse.close:type_name -> milvus.proto.streaming.CloseProducerResponse - 40, // 41: milvus.proto.streaming.ProduceMessageResponse.result:type_name -> milvus.proto.streaming.ProduceMessageResponseResult - 32, // 42: milvus.proto.streaming.ProduceMessageResponse.error:type_name -> milvus.proto.streaming.StreamingError - 69, // 43: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID - 71, // 44: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext - 72, // 45: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any - 46, // 46: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumer:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest - 45, // 47: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumers:type_name -> milvus.proto.streaming.CreateVChannelConsumersRequest - 49, // 48: milvus.proto.streaming.ConsumeRequest.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerRequest - 43, // 49: milvus.proto.streaming.ConsumeRequest.close:type_name -> milvus.proto.streaming.CloseConsumerRequest - 4, // 50: milvus.proto.streaming.CreateConsumerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 46, // 51: milvus.proto.streaming.CreateVChannelConsumersRequest.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest - 27, // 52: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_policy:type_name -> milvus.proto.streaming.DeliverPolicy - 28, // 53: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_filters:type_name -> milvus.proto.streaming.DeliverFilter - 48, // 54: milvus.proto.streaming.CreateVChannelConsumersResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse - 32, // 55: milvus.proto.streaming.CreateVChannelConsumerResponse.error:type_name -> milvus.proto.streaming.StreamingError - 52, // 56: milvus.proto.streaming.ConsumeResponse.create:type_name -> milvus.proto.streaming.CreateConsumerResponse - 53, // 57: milvus.proto.streaming.ConsumeResponse.consume:type_name -> milvus.proto.streaming.ConsumeMessageReponse - 48, // 58: milvus.proto.streaming.ConsumeResponse.create_vchannel:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse - 47, // 59: milvus.proto.streaming.ConsumeResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumersResponse - 50, // 60: milvus.proto.streaming.ConsumeResponse.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerResponse - 54, // 61: milvus.proto.streaming.ConsumeResponse.close:type_name -> milvus.proto.streaming.CloseConsumerResponse - 73, // 62: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage - 4, // 63: milvus.proto.streaming.StreamingNodeManagerAssignRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 4, // 64: milvus.proto.streaming.StreamingNodeManagerRemoveRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 60, // 65: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse.balance_attributes:type_name -> milvus.proto.streaming.StreamingNodeBalanceAttributes - 3, // 66: milvus.proto.streaming.SegmentAssignmentMeta.state:type_name -> milvus.proto.streaming.SegmentAssignmentState - 63, // 67: milvus.proto.streaming.SegmentAssignmentMeta.stat:type_name -> milvus.proto.streaming.SegmentAssignmentStat - 69, // 68: milvus.proto.streaming.WALCheckpoint.messageID:type_name -> milvus.proto.messages.MessageID - 40, // 69: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult - 74, // 70: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 9, // 71: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest - 11, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:input_type -> milvus.proto.streaming.BroadcastAckRequest - 13, // 73: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:input_type -> milvus.proto.streaming.BroadcastWatchRequest - 19, // 74: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest - 33, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest - 42, // 76: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest - 55, // 77: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest - 57, // 78: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest - 59, // 79: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest - 75, // 80: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 10, // 81: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse - 12, // 82: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:output_type -> milvus.proto.streaming.BroadcastAckResponse - 16, // 83: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:output_type -> milvus.proto.streaming.BroadcastWatchResponse - 22, // 84: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse - 37, // 85: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse - 51, // 86: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse - 56, // 87: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse - 58, // 88: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse - 61, // 89: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse - 80, // [80:90] is the sub-list for method output_type - 70, // [70:80] is the sub-list for method input_type - 70, // [70:70] is the sub-list for extension type_name - 70, // [70:70] is the sub-list for extension extendee - 0, // [0:70] is the sub-list for field type_name + 60, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message + 59, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry + 14, // 9: milvus.proto.streaming.AssignmentDiscoverRequest.report_error:type_name -> milvus.proto.streaming.ReportAssignmentErrorRequest + 15, // 10: milvus.proto.streaming.AssignmentDiscoverRequest.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverRequest + 4, // 11: milvus.proto.streaming.ReportAssignmentErrorRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 26, // 12: milvus.proto.streaming.ReportAssignmentErrorRequest.err:type_name -> milvus.proto.streaming.StreamingError + 17, // 13: milvus.proto.streaming.AssignmentDiscoverResponse.full_assignment:type_name -> milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion + 18, // 14: milvus.proto.streaming.AssignmentDiscoverResponse.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverResponse + 7, // 15: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.version:type_name -> milvus.proto.streaming.VersionPair + 20, // 16: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.assignments:type_name -> milvus.proto.streaming.StreamingNodeAssignment + 19, // 17: milvus.proto.streaming.StreamingNodeAssignment.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 4, // 18: milvus.proto.streaming.StreamingNodeAssignment.channels:type_name -> milvus.proto.streaming.PChannelInfo + 61, // 19: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty + 61, // 20: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty + 62, // 21: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID + 62, // 22: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID + 23, // 23: milvus.proto.streaming.DeliverFilter.time_tick_gt:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGT + 24, // 24: milvus.proto.streaming.DeliverFilter.time_tick_gte:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGTE + 25, // 25: milvus.proto.streaming.DeliverFilter.message_type:type_name -> milvus.proto.streaming.DeliverFilterMessageType + 63, // 26: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType + 2, // 27: milvus.proto.streaming.StreamingError.code:type_name -> milvus.proto.streaming.StreamingCode + 29, // 28: milvus.proto.streaming.ProduceRequest.produce:type_name -> milvus.proto.streaming.ProduceMessageRequest + 30, // 29: milvus.proto.streaming.ProduceRequest.close:type_name -> milvus.proto.streaming.CloseProducerRequest + 4, // 30: milvus.proto.streaming.CreateProducerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 60, // 31: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message + 32, // 32: milvus.proto.streaming.ProduceResponse.create:type_name -> milvus.proto.streaming.CreateProducerResponse + 33, // 33: milvus.proto.streaming.ProduceResponse.produce:type_name -> milvus.proto.streaming.ProduceMessageResponse + 35, // 34: milvus.proto.streaming.ProduceResponse.close:type_name -> milvus.proto.streaming.CloseProducerResponse + 34, // 35: milvus.proto.streaming.ProduceMessageResponse.result:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 26, // 36: milvus.proto.streaming.ProduceMessageResponse.error:type_name -> milvus.proto.streaming.StreamingError + 62, // 37: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID + 64, // 38: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext + 65, // 39: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any + 40, // 40: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumer:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest + 39, // 41: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumers:type_name -> milvus.proto.streaming.CreateVChannelConsumersRequest + 43, // 42: milvus.proto.streaming.ConsumeRequest.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerRequest + 37, // 43: milvus.proto.streaming.ConsumeRequest.close:type_name -> milvus.proto.streaming.CloseConsumerRequest + 4, // 44: milvus.proto.streaming.CreateConsumerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 40, // 45: milvus.proto.streaming.CreateVChannelConsumersRequest.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest + 21, // 46: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_policy:type_name -> milvus.proto.streaming.DeliverPolicy + 22, // 47: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_filters:type_name -> milvus.proto.streaming.DeliverFilter + 42, // 48: milvus.proto.streaming.CreateVChannelConsumersResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse + 26, // 49: milvus.proto.streaming.CreateVChannelConsumerResponse.error:type_name -> milvus.proto.streaming.StreamingError + 46, // 50: milvus.proto.streaming.ConsumeResponse.create:type_name -> milvus.proto.streaming.CreateConsumerResponse + 47, // 51: milvus.proto.streaming.ConsumeResponse.consume:type_name -> milvus.proto.streaming.ConsumeMessageReponse + 42, // 52: milvus.proto.streaming.ConsumeResponse.create_vchannel:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse + 41, // 53: milvus.proto.streaming.ConsumeResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumersResponse + 44, // 54: milvus.proto.streaming.ConsumeResponse.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerResponse + 48, // 55: milvus.proto.streaming.ConsumeResponse.close:type_name -> milvus.proto.streaming.CloseConsumerResponse + 66, // 56: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage + 4, // 57: milvus.proto.streaming.StreamingNodeManagerAssignRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 4, // 58: milvus.proto.streaming.StreamingNodeManagerRemoveRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo + 54, // 59: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse.balance_attributes:type_name -> milvus.proto.streaming.StreamingNodeBalanceAttributes + 3, // 60: milvus.proto.streaming.SegmentAssignmentMeta.state:type_name -> milvus.proto.streaming.SegmentAssignmentState + 57, // 61: milvus.proto.streaming.SegmentAssignmentMeta.stat:type_name -> milvus.proto.streaming.SegmentAssignmentStat + 62, // 62: milvus.proto.streaming.WALCheckpoint.messageID:type_name -> milvus.proto.messages.MessageID + 34, // 63: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 67, // 64: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 9, // 65: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest + 11, // 66: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:input_type -> milvus.proto.streaming.BroadcastAckRequest + 13, // 67: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest + 27, // 68: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest + 36, // 69: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest + 49, // 70: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest + 51, // 71: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest + 53, // 72: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + 68, // 73: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 10, // 74: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse + 12, // 75: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:output_type -> milvus.proto.streaming.BroadcastAckResponse + 16, // 76: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse + 31, // 77: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse + 45, // 78: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse + 50, // 79: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse + 52, // 80: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse + 55, // 81: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + 73, // [73:82] is the sub-list for method output_type + 64, // [64:73] is the sub-list for method input_type + 64, // [64:64] is the sub-list for extension type_name + 64, // [64:64] is the sub-list for extension extendee + 0, // [0:64] is the sub-list for field type_name } func init() { file_streaming_proto_init() } @@ -4928,78 +4532,6 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastWatchRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_streaming_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastCreateEventWatchRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_streaming_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseBroadcastWatchRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_streaming_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastWatchResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_streaming_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastEventWatchResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_streaming_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseBroadcastWatchResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_streaming_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*AssignmentDiscoverRequest); i { case 0: return &v.state @@ -5011,7 +4543,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReportAssignmentErrorRequest); i { case 0: return &v.state @@ -5023,7 +4555,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseAssignmentDiscoverRequest); i { case 0: return &v.state @@ -5035,7 +4567,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*AssignmentDiscoverResponse); i { case 0: return &v.state @@ -5047,7 +4579,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*FullStreamingNodeAssignmentWithVersion); i { case 0: return &v.state @@ -5059,7 +4591,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseAssignmentDiscoverResponse); i { case 0: return &v.state @@ -5071,7 +4603,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeInfo); i { case 0: return &v.state @@ -5083,7 +4615,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeAssignment); i { case 0: return &v.state @@ -5095,7 +4627,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeliverPolicy); i { case 0: return &v.state @@ -5107,7 +4639,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeliverFilter); i { case 0: return &v.state @@ -5119,7 +4651,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeliverFilterTimeTickGT); i { case 0: return &v.state @@ -5131,7 +4663,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeliverFilterTimeTickGTE); i { case 0: return &v.state @@ -5143,7 +4675,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeliverFilterMessageType); i { case 0: return &v.state @@ -5155,7 +4687,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingError); i { case 0: return &v.state @@ -5167,7 +4699,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ProduceRequest); i { case 0: return &v.state @@ -5179,7 +4711,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateProducerRequest); i { case 0: return &v.state @@ -5191,7 +4723,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ProduceMessageRequest); i { case 0: return &v.state @@ -5203,7 +4735,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseProducerRequest); i { case 0: return &v.state @@ -5215,7 +4747,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ProduceResponse); i { case 0: return &v.state @@ -5227,7 +4759,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateProducerResponse); i { case 0: return &v.state @@ -5239,7 +4771,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ProduceMessageResponse); i { case 0: return &v.state @@ -5251,7 +4783,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ProduceMessageResponseResult); i { case 0: return &v.state @@ -5263,7 +4795,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseProducerResponse); i { case 0: return &v.state @@ -5275,7 +4807,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ConsumeRequest); i { case 0: return &v.state @@ -5287,7 +4819,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseConsumerRequest); i { case 0: return &v.state @@ -5299,7 +4831,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateConsumerRequest); i { case 0: return &v.state @@ -5311,7 +4843,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateVChannelConsumersRequest); i { case 0: return &v.state @@ -5323,7 +4855,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateVChannelConsumerRequest); i { case 0: return &v.state @@ -5335,7 +4867,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateVChannelConsumersResponse); i { case 0: return &v.state @@ -5347,7 +4879,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateVChannelConsumerResponse); i { case 0: return &v.state @@ -5359,7 +4891,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseVChannelConsumerRequest); i { case 0: return &v.state @@ -5371,7 +4903,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseVChannelConsumerResponse); i { case 0: return &v.state @@ -5383,7 +4915,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ConsumeResponse); i { case 0: return &v.state @@ -5395,7 +4927,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateConsumerResponse); i { case 0: return &v.state @@ -5407,7 +4939,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ConsumeMessageReponse); i { case 0: return &v.state @@ -5419,7 +4951,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CloseConsumerResponse); i { case 0: return &v.state @@ -5431,7 +4963,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeManagerAssignRequest); i { case 0: return &v.state @@ -5443,7 +4975,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeManagerAssignResponse); i { case 0: return &v.state @@ -5455,7 +4987,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeManagerRemoveRequest); i { case 0: return &v.state @@ -5467,7 +4999,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeManagerRemoveResponse); i { case 0: return &v.state @@ -5479,7 +5011,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeManagerCollectStatusRequest); i { case 0: return &v.state @@ -5491,7 +5023,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeBalanceAttributes); i { case 0: return &v.state @@ -5503,7 +5035,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StreamingNodeManagerCollectStatusResponse); i { case 0: return &v.state @@ -5515,7 +5047,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SegmentAssignmentMeta); i { case 0: return &v.state @@ -5527,7 +5059,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SegmentAssignmentStat); i { case 0: return &v.state @@ -5539,7 +5071,7 @@ func file_streaming_proto_init() { return nil } } - file_streaming_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + file_streaming_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WALCheckpoint); i { case 0: return &v.state @@ -5553,56 +5085,48 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[9].OneofWrappers = []interface{}{ - (*BroadcastWatchRequest_CreateEventWatch)(nil), - (*BroadcastWatchRequest_Close)(nil), - } - file_streaming_proto_msgTypes[12].OneofWrappers = []interface{}{ - (*BroadcastWatchResponse_EventDone)(nil), - (*BroadcastWatchResponse_Close)(nil), - } - file_streaming_proto_msgTypes[15].OneofWrappers = []interface{}{ (*AssignmentDiscoverRequest_ReportError)(nil), (*AssignmentDiscoverRequest_Close)(nil), } - file_streaming_proto_msgTypes[18].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[12].OneofWrappers = []interface{}{ (*AssignmentDiscoverResponse_FullAssignment)(nil), (*AssignmentDiscoverResponse_Close)(nil), } - file_streaming_proto_msgTypes[23].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[17].OneofWrappers = []interface{}{ (*DeliverPolicy_All)(nil), (*DeliverPolicy_Latest)(nil), (*DeliverPolicy_StartFrom)(nil), (*DeliverPolicy_StartAfter)(nil), } - file_streaming_proto_msgTypes[24].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[18].OneofWrappers = []interface{}{ (*DeliverFilter_TimeTickGt)(nil), (*DeliverFilter_TimeTickGte)(nil), (*DeliverFilter_MessageType)(nil), } - file_streaming_proto_msgTypes[29].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[23].OneofWrappers = []interface{}{ (*ProduceRequest_Produce)(nil), (*ProduceRequest_Close)(nil), } - file_streaming_proto_msgTypes[33].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[27].OneofWrappers = []interface{}{ (*ProduceResponse_Create)(nil), (*ProduceResponse_Produce)(nil), (*ProduceResponse_Close)(nil), } - file_streaming_proto_msgTypes[35].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[29].OneofWrappers = []interface{}{ (*ProduceMessageResponse_Result)(nil), (*ProduceMessageResponse_Error)(nil), } - file_streaming_proto_msgTypes[38].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[32].OneofWrappers = []interface{}{ (*ConsumeRequest_CreateVchannelConsumer)(nil), (*ConsumeRequest_CreateVchannelConsumers)(nil), (*ConsumeRequest_CloseVchannel)(nil), (*ConsumeRequest_Close)(nil), } - file_streaming_proto_msgTypes[44].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[38].OneofWrappers = []interface{}{ (*CreateVChannelConsumerResponse_ConsumerId)(nil), (*CreateVChannelConsumerResponse_Error)(nil), } - file_streaming_proto_msgTypes[47].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[41].OneofWrappers = []interface{}{ (*ConsumeResponse_Create)(nil), (*ConsumeResponse_Consume)(nil), (*ConsumeResponse_CreateVchannel)(nil), @@ -5616,7 +5140,7 @@ func file_streaming_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_streaming_proto_rawDesc, NumEnums: 4, - NumMessages: 62, + NumMessages: 56, NumExtensions: 0, NumServices: 5, }, diff --git a/pkg/proto/streamingpb/streaming_grpc.pb.go b/pkg/proto/streamingpb/streaming_grpc.pb.go index 6540e92a20..cf1ed5e9d0 100644 --- a/pkg/proto/streamingpb/streaming_grpc.pb.go +++ b/pkg/proto/streamingpb/streaming_grpc.pb.go @@ -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", }