diff --git a/internal/.mockery.yaml b/internal/.mockery.yaml index 8804105c23322..537a0b6f395ec 100644 --- a/internal/.mockery.yaml +++ b/internal/.mockery.yaml @@ -15,10 +15,14 @@ packages: github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster: interfaces: AppendOperator: + Watcher: github.com/milvus-io/milvus/internal/streamingcoord/client: interfaces: Client: BroadcastService: + github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast: + interfaces: + Watcher: github.com/milvus-io/milvus/internal/streamingnode/client/manager: interfaces: ManagerClient: diff --git a/internal/coordinator/coordclient/registry.go b/internal/coordinator/coordclient/registry.go index efb0dbed2122e..7602d1c710b15 100644 --- a/internal/coordinator/coordclient/registry.go +++ b/internal/coordinator/coordclient/registry.go @@ -58,9 +58,6 @@ func EnableLocalClientRole(cfg *LocalClientRoleConfig) { // RegisterQueryCoordServer register query coord server func RegisterQueryCoordServer(server querypb.QueryCoordServer) { - if !enableLocal.EnableQueryCoord { - return - } newLocalClient := grpcclient.NewLocalGRPCClient(&querypb.QueryCoord_ServiceDesc, server, querypb.NewQueryCoordClient) glocalClient.queryCoordClient.Set(&nopCloseQueryCoordClient{newLocalClient}) log.Ctx(context.TODO()).Info("register query coord server", zap.Any("enableLocalClient", enableLocal)) @@ -68,9 +65,6 @@ func RegisterQueryCoordServer(server querypb.QueryCoordServer) { // RegsterDataCoordServer register data coord server func RegisterDataCoordServer(server datapb.DataCoordServer) { - if !enableLocal.EnableDataCoord { - return - } newLocalClient := grpcclient.NewLocalGRPCClient(&datapb.DataCoord_ServiceDesc, server, datapb.NewDataCoordClient) glocalClient.dataCoordClient.Set(&nopCloseDataCoordClient{newLocalClient}) log.Ctx(context.TODO()).Info("register data coord server", zap.Any("enableLocalClient", enableLocal)) @@ -78,9 +72,6 @@ func RegisterDataCoordServer(server datapb.DataCoordServer) { // RegisterRootCoordServer register root coord server func RegisterRootCoordServer(server rootcoordpb.RootCoordServer) { - if !enableLocal.EnableRootCoord { - return - } newLocalClient := grpcclient.NewLocalGRPCClient(&rootcoordpb.RootCoord_ServiceDesc, server, rootcoordpb.NewRootCoordClient) glocalClient.rootCoordClient.Set(&nopCloseRootCoordClient{newLocalClient}) log.Ctx(context.TODO()).Info("register root coord server", zap.Any("enableLocalClient", enableLocal)) diff --git a/internal/coordinator/coordclient/registry_test.go b/internal/coordinator/coordclient/registry_test.go index df36ba8c25ae2..d796c11a92e0b 100644 --- a/internal/coordinator/coordclient/registry_test.go +++ b/internal/coordinator/coordclient/registry_test.go @@ -34,9 +34,10 @@ func TestRegistry(t *testing.T) { RegisterRootCoordServer(&rootcoordpb.UnimplementedRootCoordServer{}) RegisterDataCoordServer(&datapb.UnimplementedDataCoordServer{}) RegisterQueryCoordServer(&querypb.UnimplementedQueryCoordServer{}) - assert.False(t, glocalClient.dataCoordClient.Ready()) - assert.False(t, glocalClient.queryCoordClient.Ready()) - assert.False(t, glocalClient.rootCoordClient.Ready()) + assert.True(t, glocalClient.dataCoordClient.Ready()) + assert.True(t, glocalClient.queryCoordClient.Ready()) + assert.True(t, glocalClient.rootCoordClient.Ready()) + ResetRegistration() enableLocal = &LocalClientRoleConfig{} diff --git a/internal/coordinator/coordclient/test_utility.go b/internal/coordinator/coordclient/test_utility.go new file mode 100644 index 0000000000000..b5832cdf2f6cb --- /dev/null +++ b/internal/coordinator/coordclient/test_utility.go @@ -0,0 +1,32 @@ +//go:build test +// +build test + +package coordclient + +import ( + "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +// ResetRegistration resets the global local client to initial state. +// This function is only used in test. +func ResetRegistration() { + glocalClient = &localClient{ + queryCoordClient: syncutil.NewFuture[types.QueryCoordClient](), + dataCoordClient: syncutil.NewFuture[types.DataCoordClient](), + rootCoordClient: syncutil.NewFuture[types.RootCoordClient](), + } +} + +// ResetQueryCoordRegistration resets the query coord client to initial state. +func ResetQueryCoordRegistration() { + glocalClient.queryCoordClient = syncutil.NewFuture[types.QueryCoordClient]() +} + +func ResetRootCoordRegistration() { + glocalClient.rootCoordClient = syncutil.NewFuture[types.RootCoordClient]() +} + +func ResetDataCoordRegistration() { + glocalClient.dataCoordClient = syncutil.NewFuture[types.DataCoordClient]() +} diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index 004115a63e932..d55539ff626db 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/proto/indexpb" @@ -38,6 +39,7 @@ import ( func Test_NewServer(t *testing.T) { paramtable.Init() + coordclient.ResetRegistration() ctx := context.Background() mockDataCoord := mocks.NewMockDataCoord(t) @@ -335,6 +337,7 @@ func Test_Run(t *testing.T) { t.Run("test run success", func(t *testing.T) { parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { @@ -370,6 +373,7 @@ func Test_Run(t *testing.T) { paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, "etcd") t.Run("test init error", func(t *testing.T) { + coordclient.ResetRegistration() ctx := context.Background() server, err := NewServer(ctx, nil) assert.NotNil(t, server) @@ -390,6 +394,7 @@ func Test_Run(t *testing.T) { }) t.Run("test register error", func(t *testing.T) { + coordclient.ResetRegistration() ctx := context.Background() server, err := NewServer(ctx, nil) assert.NoError(t, err) @@ -411,6 +416,7 @@ func Test_Run(t *testing.T) { }) t.Run("test start error", func(t *testing.T) { + coordclient.ResetRegistration() ctx := context.Background() server, err := NewServer(ctx, nil) assert.NoError(t, err) @@ -433,6 +439,7 @@ func Test_Run(t *testing.T) { }) t.Run("test stop error", func(t *testing.T) { + coordclient.ResetRegistration() ctx := context.Background() server, err := NewServer(ctx, nil) assert.NoError(t, err) diff --git a/internal/distributed/querycoord/service_test.go b/internal/distributed/querycoord/service_test.go index 1119b038ed3a7..5d5ed930592fe 100644 --- a/internal/distributed/querycoord/service_test.go +++ b/internal/distributed/querycoord/service_test.go @@ -28,6 +28,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/proto/querypb" @@ -46,6 +47,7 @@ func TestMain(m *testing.M) { func Test_NewServer(t *testing.T) { parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { @@ -358,6 +360,7 @@ func Test_NewServer(t *testing.T) { func TestServer_Run1(t *testing.T) { parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) t.Skip() ctx := context.Background() diff --git a/internal/distributed/rootcoord/service_test.go b/internal/distributed/rootcoord/service_test.go index f475396d70094..9e7b5115b7b45 100644 --- a/internal/distributed/rootcoord/service_test.go +++ b/internal/distributed/rootcoord/service_test.go @@ -31,6 +31,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/rootcoord" "github.com/milvus-io/milvus/internal/types" @@ -121,6 +122,7 @@ func TestRun(t *testing.T) { paramtable.Init() parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { @@ -228,6 +230,7 @@ func TestServerRun_DataCoordClientInitErr(t *testing.T) { paramtable.Init() parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { @@ -258,6 +261,7 @@ func TestServerRun_DataCoordClientStartErr(t *testing.T) { paramtable.Init() parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { @@ -288,6 +292,7 @@ func TestServerRun_QueryCoordClientInitErr(t *testing.T) { paramtable.Init() parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { @@ -318,6 +323,7 @@ func TestServer_QueryCoordClientStartErr(t *testing.T) { paramtable.Init() parameters := []string{"tikv", "etcd"} for _, v := range parameters { + coordclient.ResetRegistration() paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v) ctx := context.Background() getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) { diff --git a/internal/distributed/streaming/append.go b/internal/distributed/streaming/append.go index b4193d8b94446..59616fa3da48d 100644 --- a/internal/distributed/streaming/append.go +++ b/internal/distributed/streaming/append.go @@ -17,12 +17,6 @@ func (w *walAccesserImpl) appendToWAL(ctx context.Context, msg message.MutableMe return p.Produce(ctx, msg) } -func (w *walAccesserImpl) broadcastToWAL(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { - // The broadcast operation will be sent to the coordinator. - // The coordinator will dispatch the message to all the vchannels with an eventual consistency guarantee. - return w.streamingCoordClient.Broadcast().Broadcast(ctx, msg) -} - // createOrGetProducer creates or get a producer. // vchannel in same pchannel can share the same producer. func (w *walAccesserImpl) getProducer(pchannel string) *producer.ResumableProducer { diff --git a/internal/distributed/streaming/broadcast.go b/internal/distributed/streaming/broadcast.go new file mode 100644 index 0000000000000..7d9bcea739488 --- /dev/null +++ b/internal/distributed/streaming/broadcast.go @@ -0,0 +1,52 @@ +package streaming + +import ( + "context" + + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +var _ Broadcast = broadcast{} + +type broadcast struct { + *walAccesserImpl +} + +func (b broadcast) Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + assertValidBroadcastMessage(msg) + if !b.lifetime.Add(typeutil.LifetimeStateWorking) { + return nil, ErrWALAccesserClosed + } + defer b.lifetime.Done() + + // The broadcast operation will be sent to the coordinator. + // The coordinator will dispatch the message to all the vchannels with an eventual consistency guarantee. + return b.streamingCoordClient.Broadcast().Broadcast(ctx, msg) +} + +func (b broadcast) Ack(ctx context.Context, req types.BroadcastAckRequest) error { + if !b.lifetime.Add(typeutil.LifetimeStateWorking) { + return ErrWALAccesserClosed + } + defer b.lifetime.Done() + + 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 df54d7c95ecc9..8db663afdc1cb 100644 --- a/internal/distributed/streaming/streaming.go +++ b/internal/distributed/streaming/streaming.go @@ -84,16 +84,20 @@ type WALAccesser interface { // WALName returns the name of the wal. WALName() string - // Txn returns a transaction for writing records to the log. + // Txn returns a transaction for writing records to one vchannel. + // It promises the atomicity written of the messages. // Once the txn is returned, the Commit or Rollback operation must be called once, otherwise resource leak on wal. Txn(ctx context.Context, opts TxnOption) (Txn, error) // RawAppend writes a records to the log. RawAppend(ctx context.Context, msgs message.MutableMessage, opts ...AppendOption) (*types.AppendResult, error) - // BroadcastAppend sends a broadcast message to all target vchannels. - // BroadcastAppend guarantees the atomicity written of the messages and eventual consistency. - BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) + // Broadcast returns a broadcast service of wal. + // Broadcast support cross-vchannel message broadcast. + // It promises the atomicity written of the messages and eventual consistency. + // And the broadcasted message must be acked cat consuming-side, otherwise resource leak on broadcast. + // Broadcast also support the resource-key to achieve a resource-exclusive acquirsion. + Broadcast() Broadcast // Read returns a scanner for reading records from the wal. Read(ctx context.Context, opts ReadOption) Scanner @@ -103,15 +107,33 @@ type WALAccesser interface { // If the messages is belong to one vchannel, it will be sent as a transaction. // Otherwise, it will be sent as individual messages. // !!! This function do not promise the atomicity and deliver order of the messages appending. - // TODO: Remove after we support cross-wal txn. AppendMessages(ctx context.Context, msgs ...message.MutableMessage) AppendResponses // AppendMessagesWithOption appends messages to the wal with the given option. // Same with AppendMessages, but with the given option. - // TODO: Remove after we support cross-wal txn. AppendMessagesWithOption(ctx context.Context, opts AppendOption, msgs ...message.MutableMessage) AppendResponses } +// Broadcast is the interface for writing broadcast message into the wal. +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. + 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. type Txn interface { // Append writes a record to the log. diff --git a/internal/distributed/streaming/streaming_test.go b/internal/distributed/streaming/streaming_test.go index 0da14f0923512..f9f5150d12479 100644 --- a/internal/distributed/streaming/streaming_test.go +++ b/internal/distributed/streaming/streaming_test.go @@ -6,11 +6,16 @@ import ( "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/util/streamingutil" + "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/options" + "github.com/milvus-io/milvus/pkg/streaming/util/types" _ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/pulsar" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -20,13 +25,101 @@ var vChannels = []string{ "by-dev-rootcoord-dml_5", } +var collectionName = "test" + func TestMain(m *testing.M) { + streamingutil.SetStreamingServiceEnabled() paramtable.Init() m.Run() } +func TestStreamingBroadcast(t *testing.T) { + t.Skip("cat not running without streaming service at background") + streamingutil.SetStreamingServiceEnabled() + 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, + PartitionIds: []int64{1, 2, 3}, + }). + WithBody(&msgpb.CreateCollectionRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_CreateCollection, + Timestamp: 1, + }, + CollectionID: 1, + CollectionName: collectionName, + }). + WithBroadcast(vChannels, message.NewCollectionNameResourceKey(collectionName)). + BuildBroadcast() + + resp, err := streaming.WAL().Broadcast().Append(context.Background(), msg) + assert.NoError(t, err) + assert.NotNil(t, resp) + t.Logf("CreateCollection: %+v\t%+v\n", resp, err) + + // repeated broadcast with same resource key should be rejected + resp2, err := streaming.WAL().Broadcast().Append(context.Background(), msg) + 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) { - t.Skip() + t.Skip("cat not running without streaming service at background") + streamingutil.SetStreamingServiceEnabled() streaming.Init() defer streaming.Release() msg, _ := message.NewCreateCollectionMessageBuilderV1(). @@ -39,12 +132,13 @@ func TestStreamingProduce(t *testing.T) { MsgType: commonpb.MsgType_CreateCollection, Timestamp: 1, }, - CollectionID: 1, + CollectionID: 1, + CollectionName: collectionName, }). WithBroadcast(vChannels). BuildBroadcast() - resp, err := streaming.WAL().BroadcastAppend(context.Background(), msg) + resp, err := streaming.WAL().Broadcast().Append(context.Background(), msg) t.Logf("CreateCollection: %+v\t%+v\n", resp, err) for i := 0; i < 500; i++ { @@ -101,12 +195,12 @@ func TestStreamingProduce(t *testing.T) { }). WithBroadcast(vChannels). BuildBroadcast() - resp, err = streaming.WAL().BroadcastAppend(context.Background(), msg) + resp, err = streaming.WAL().Broadcast().Append(context.Background(), msg) t.Logf("DropCollection: %+v\t%+v\n", resp, err) } func TestStreamingConsume(t *testing.T) { - t.Skip() + t.Skip("cat not running without streaming service at background") streaming.Init() defer streaming.Release() ch := make(message.ChanMessageHandler, 10) diff --git a/internal/distributed/streaming/wal.go b/internal/distributed/streaming/wal.go index ee585f8bd06ac..d0937d8f29657 100644 --- a/internal/distributed/streaming/wal.go +++ b/internal/distributed/streaming/wal.go @@ -76,16 +76,6 @@ func (w *walAccesserImpl) RawAppend(ctx context.Context, msg message.MutableMess return w.appendToWAL(ctx, msg) } -func (w *walAccesserImpl) BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { - assertValidBroadcastMessage(msg) - if !w.lifetime.Add(typeutil.LifetimeStateWorking) { - return nil, ErrWALAccesserClosed - } - defer w.lifetime.Done() - - return w.broadcastToWAL(ctx, msg) -} - // Read returns a scanner for reading records from the wal. func (w *walAccesserImpl) Read(_ context.Context, opts ReadOption) Scanner { if !w.lifetime.Add(typeutil.LifetimeStateWorking) { @@ -109,6 +99,11 @@ func (w *walAccesserImpl) Read(_ context.Context, opts ReadOption) Scanner { return rc } +// Broadcast returns a broadcast for broadcasting records to the wal. +func (w *walAccesserImpl) Broadcast() Broadcast { + return broadcast{w} +} + func (w *walAccesserImpl) Txn(ctx context.Context, opts TxnOption) (Txn, error) { if !w.lifetime.Add(typeutil.LifetimeStateWorking) { return nil, ErrWALAccesserClosed diff --git a/internal/distributed/streaming/wal_test.go b/internal/distributed/streaming/wal_test.go index a850b9cce3a07..1298332780251 100644 --- a/internal/distributed/streaming/wal_test.go +++ b/internal/distributed/streaming/wal_test.go @@ -33,6 +33,7 @@ func TestWAL(t *testing.T) { broadcastServce := mock_client.NewMockBroadcastService(t) broadcastServce.EXPECT().Broadcast(mock.Anything, mock.Anything).RunAndReturn( func(ctx context.Context, bmm message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + bmm = bmm.WithBroadcastID(1) result := make(map[string]*types.AppendResult) for idx, msg := range bmm.SplitIntoMutableMessage() { result[msg.VChannel()] = &types.AppendResult{ @@ -44,6 +45,8 @@ func TestWAL(t *testing.T) { AppendResults: result, }, 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() @@ -129,17 +132,36 @@ func TestWAL(t *testing.T) { ) assert.NoError(t, resp.UnwrapFirstError()) - r, err := w.BroadcastAppend(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) + r, err := w.Broadcast().Append(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) assert.NoError(t, err) assert.Len(t, r.AppendResults, 3) + err = w.Broadcast().Ack(ctx, types.BroadcastAckRequest{BroadcastID: 1, VChannel: vChannel1}) + assert.NoError(t, err) + + err = w.Broadcast().BlockUntilResourceKeyAckAll(ctx, message.NewCollectionNameResourceKey("r1")) + assert.NoError(t, err) + + err = w.Broadcast().BlockUntilResourceKeyAckOnce(ctx, message.NewCollectionNameResourceKey("r2")) + assert.NoError(t, err) + w.Close() resp = w.AppendMessages(ctx, newInsertMessage(vChannel1)) assert.Error(t, resp.UnwrapFirstError()) - r, err = w.BroadcastAppend(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) + + r, err = w.Broadcast().Append(ctx, newBroadcastMessage([]string{vChannel1, vChannel2, vChannel3})) assert.Error(t, err) assert.Nil(t, r) + + 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/metastore/catalog.go b/internal/metastore/catalog.go index 0238cf74550a0..9fa151691449b 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -218,7 +218,7 @@ type StreamingCoordCataLog interface { // SaveBroadcastTask save the broadcast task to metastore. // Make the task recoverable after restart. // When broadcast task is done, it will be removed from metastore. - SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error + SaveBroadcastTask(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask) error } // StreamingNodeCataLog is the interface for streamingnode catalog diff --git a/internal/metastore/kv/streamingcoord/kv_catalog.go b/internal/metastore/kv/streamingcoord/kv_catalog.go index 539e0ca4daaf6..a09ee0a4f4c29 100644 --- a/internal/metastore/kv/streamingcoord/kv_catalog.go +++ b/internal/metastore/kv/streamingcoord/kv_catalog.go @@ -86,8 +86,8 @@ func (c *catalog) ListBroadcastTask(ctx context.Context) ([]*streamingpb.Broadca return infos, nil } -func (c *catalog) SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error { - key := buildBroadcastTaskPath(task.TaskId) +func (c *catalog) SaveBroadcastTask(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask) error { + key := buildBroadcastTaskPath(broadcastID) if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { return c.metaKV.Remove(ctx, key) } @@ -104,6 +104,6 @@ func buildPChannelInfoPath(name string) string { } // buildBroadcastTaskPath builds the path for broadcast task. -func buildBroadcastTaskPath(id int64) string { - return BroadcastTaskPrefix + strconv.FormatInt(id, 10) +func buildBroadcastTaskPath(id uint64) string { + return BroadcastTaskPrefix + strconv.FormatUint(id, 10) } diff --git a/internal/metastore/kv/streamingcoord/kv_catalog_test.go b/internal/metastore/kv/streamingcoord/kv_catalog_test.go index b805ccf05407b..c9434918fa75b 100644 --- a/internal/metastore/kv/streamingcoord/kv_catalog_test.go +++ b/internal/metastore/kv/streamingcoord/kv_catalog_test.go @@ -66,14 +66,12 @@ func TestCatalog(t *testing.T) { assert.Len(t, metas, 2) // BroadcastTask test - err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ - TaskId: 1, - State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + err = catalog.SaveBroadcastTask(context.Background(), 1, &streamingpb.BroadcastTask{ + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, }) assert.NoError(t, err) - err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ - TaskId: 2, - State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + err = catalog.SaveBroadcastTask(context.Background(), 2, &streamingpb.BroadcastTask{ + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, }) assert.NoError(t, err) @@ -84,9 +82,8 @@ func TestCatalog(t *testing.T) { assert.Equal(t, streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, task.State) } - err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{ - TaskId: 1, - State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE, + err = catalog.SaveBroadcastTask(context.Background(), 1, &streamingpb.BroadcastTask{ + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE, }) assert.NoError(t, err) tasks, err = catalog.ListBroadcastTask(context.Background()) @@ -116,6 +113,6 @@ func TestCatalog(t *testing.T) { Node: &streamingpb.StreamingNodeInfo{ServerId: 1}, }}) assert.Error(t, err) - err = catalog.SaveBroadcastTask(context.Background(), &streamingpb.BroadcastTask{}) + err = catalog.SaveBroadcastTask(context.Background(), 1, &streamingpb.BroadcastTask{}) assert.Error(t, err) } diff --git a/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go b/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go index eb9f7ce2d8b0a..784d0f3e7e237 100644 --- a/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go +++ b/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go @@ -27,7 +27,7 @@ func (_m *MockWALAccesser) EXPECT() *MockWALAccesser_Expecter { } // AppendMessages provides a mock function with given fields: ctx, msgs -func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses { +func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) types.AppendResponses { _va := make([]interface{}, len(msgs)) for _i := range msgs { _va[_i] = msgs[_i] @@ -41,11 +41,11 @@ func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.M panic("no return value specified for AppendMessages") } - var r0 streaming.AppendResponses - if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) streaming.AppendResponses); ok { + var r0 types.AppendResponses + if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) types.AppendResponses); ok { r0 = rf(ctx, msgs...) } else { - r0 = ret.Get(0).(streaming.AppendResponses) + r0 = ret.Get(0).(types.AppendResponses) } return r0 @@ -77,18 +77,18 @@ func (_c *MockWALAccesser_AppendMessages_Call) Run(run func(ctx context.Context, return _c } -func (_c *MockWALAccesser_AppendMessages_Call) Return(_a0 streaming.AppendResponses) *MockWALAccesser_AppendMessages_Call { +func (_c *MockWALAccesser_AppendMessages_Call) Return(_a0 types.AppendResponses) *MockWALAccesser_AppendMessages_Call { _c.Call.Return(_a0) return _c } -func (_c *MockWALAccesser_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) streaming.AppendResponses) *MockWALAccesser_AppendMessages_Call { +func (_c *MockWALAccesser_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) types.AppendResponses) *MockWALAccesser_AppendMessages_Call { _c.Call.Return(run) return _c } // AppendMessagesWithOption provides a mock function with given fields: ctx, opts, msgs -func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts streaming.AppendOption, msgs ...message.MutableMessage) streaming.AppendResponses { +func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts streaming.AppendOption, msgs ...message.MutableMessage) types.AppendResponses { _va := make([]interface{}, len(msgs)) for _i := range msgs { _va[_i] = msgs[_i] @@ -102,11 +102,11 @@ func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts st panic("no return value specified for AppendMessagesWithOption") } - var r0 streaming.AppendResponses - if rf, ok := ret.Get(0).(func(context.Context, streaming.AppendOption, ...message.MutableMessage) streaming.AppendResponses); ok { + var r0 types.AppendResponses + if rf, ok := ret.Get(0).(func(context.Context, streaming.AppendOption, ...message.MutableMessage) types.AppendResponses); ok { r0 = rf(ctx, opts, msgs...) } else { - r0 = ret.Get(0).(streaming.AppendResponses) + r0 = ret.Get(0).(types.AppendResponses) } return r0 @@ -139,71 +139,59 @@ func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Run(run func(ctx contex return _c } -func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Return(_a0 streaming.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { +func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Return(_a0 types.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { _c.Call.Return(_a0) return _c } -func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(context.Context, streaming.AppendOption, ...message.MutableMessage) streaming.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { +func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(context.Context, streaming.AppendOption, ...message.MutableMessage) types.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { _c.Call.Return(run) return _c } -// BroadcastAppend provides a mock function with given fields: ctx, msg -func (_m *MockWALAccesser) BroadcastAppend(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { - ret := _m.Called(ctx, msg) +// Broadcast provides a mock function with given fields: +func (_m *MockWALAccesser) Broadcast() streaming.Broadcast { + ret := _m.Called() if len(ret) == 0 { - panic("no return value specified for BroadcastAppend") + panic("no return value specified for Broadcast") } - var r0 *types.BroadcastAppendResult - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)); ok { - return rf(ctx, msg) - } - if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) *types.BroadcastAppendResult); ok { - r0 = rf(ctx, msg) + var r0 streaming.Broadcast + if rf, ok := ret.Get(0).(func() streaming.Broadcast); ok { + r0 = rf() } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(*types.BroadcastAppendResult) + r0 = ret.Get(0).(streaming.Broadcast) } } - if rf, ok := ret.Get(1).(func(context.Context, message.BroadcastMutableMessage) error); ok { - r1 = rf(ctx, msg) - } else { - r1 = ret.Error(1) - } - - return r0, r1 + return r0 } -// MockWALAccesser_BroadcastAppend_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastAppend' -type MockWALAccesser_BroadcastAppend_Call struct { +// MockWALAccesser_Broadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Broadcast' +type MockWALAccesser_Broadcast_Call struct { *mock.Call } -// BroadcastAppend is a helper method to define mock.On call -// - ctx context.Context -// - msg message.BroadcastMutableMessage -func (_e *MockWALAccesser_Expecter) BroadcastAppend(ctx interface{}, msg interface{}) *MockWALAccesser_BroadcastAppend_Call { - return &MockWALAccesser_BroadcastAppend_Call{Call: _e.mock.On("BroadcastAppend", ctx, msg)} +// Broadcast is a helper method to define mock.On call +func (_e *MockWALAccesser_Expecter) Broadcast() *MockWALAccesser_Broadcast_Call { + return &MockWALAccesser_Broadcast_Call{Call: _e.mock.On("Broadcast")} } -func (_c *MockWALAccesser_BroadcastAppend_Call) Run(run func(ctx context.Context, msg message.BroadcastMutableMessage)) *MockWALAccesser_BroadcastAppend_Call { +func (_c *MockWALAccesser_Broadcast_Call) Run(run func()) *MockWALAccesser_Broadcast_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(message.BroadcastMutableMessage)) + run() }) return _c } -func (_c *MockWALAccesser_BroadcastAppend_Call) Return(_a0 *types.BroadcastAppendResult, _a1 error) *MockWALAccesser_BroadcastAppend_Call { - _c.Call.Return(_a0, _a1) +func (_c *MockWALAccesser_Broadcast_Call) Return(_a0 streaming.Broadcast) *MockWALAccesser_Broadcast_Call { + _c.Call.Return(_a0) return _c } -func (_c *MockWALAccesser_BroadcastAppend_Call) RunAndReturn(run func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)) *MockWALAccesser_BroadcastAppend_Call { +func (_c *MockWALAccesser_Broadcast_Call) RunAndReturn(run func() streaming.Broadcast) *MockWALAccesser_Broadcast_Call { _c.Call.Return(run) return _c } diff --git a/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go b/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go index 27e603960ad82..17a060099269a 100644 --- a/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go +++ b/internal/mocks/mock_metastore/mock_StreamingCoordCataLog.go @@ -139,17 +139,17 @@ func (_c *MockStreamingCoordCataLog_ListPChannel_Call) RunAndReturn(run func(con return _c } -// SaveBroadcastTask provides a mock function with given fields: ctx, task -func (_m *MockStreamingCoordCataLog) SaveBroadcastTask(ctx context.Context, task *streamingpb.BroadcastTask) error { - ret := _m.Called(ctx, task) +// SaveBroadcastTask provides a mock function with given fields: ctx, broadcastID, task +func (_m *MockStreamingCoordCataLog) SaveBroadcastTask(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask) error { + ret := _m.Called(ctx, broadcastID, task) if len(ret) == 0 { panic("no return value specified for SaveBroadcastTask") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastTask) error); ok { - r0 = rf(ctx, task) + if rf, ok := ret.Get(0).(func(context.Context, uint64, *streamingpb.BroadcastTask) error); ok { + r0 = rf(ctx, broadcastID, task) } else { r0 = ret.Error(0) } @@ -164,14 +164,15 @@ type MockStreamingCoordCataLog_SaveBroadcastTask_Call struct { // SaveBroadcastTask is a helper method to define mock.On call // - ctx context.Context +// - broadcastID uint64 // - task *streamingpb.BroadcastTask -func (_e *MockStreamingCoordCataLog_Expecter) SaveBroadcastTask(ctx interface{}, task interface{}) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { - return &MockStreamingCoordCataLog_SaveBroadcastTask_Call{Call: _e.mock.On("SaveBroadcastTask", ctx, task)} +func (_e *MockStreamingCoordCataLog_Expecter) SaveBroadcastTask(ctx interface{}, broadcastID interface{}, task interface{}) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { + return &MockStreamingCoordCataLog_SaveBroadcastTask_Call{Call: _e.mock.On("SaveBroadcastTask", ctx, broadcastID, task)} } -func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Run(run func(ctx context.Context, task *streamingpb.BroadcastTask)) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { +func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Run(run func(ctx context.Context, broadcastID uint64, task *streamingpb.BroadcastTask)) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(*streamingpb.BroadcastTask)) + run(args[0].(context.Context), args[1].(uint64), args[2].(*streamingpb.BroadcastTask)) }) return _c } @@ -181,7 +182,7 @@ func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) Return(_a0 error) *M return _c } -func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastTask) error) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { +func (_c *MockStreamingCoordCataLog_SaveBroadcastTask_Call) RunAndReturn(run func(context.Context, uint64, *streamingpb.BroadcastTask) error) *MockStreamingCoordCataLog_SaveBroadcastTask_Call { _c.Call.Return(run) return _c } diff --git a/internal/mocks/streamingcoord/client/mock_broadcast/mock_Watcher.go b/internal/mocks/streamingcoord/client/mock_broadcast/mock_Watcher.go new file mode 100644 index 0000000000000..273d5f1713b87 --- /dev/null +++ b/internal/mocks/streamingcoord/client/mock_broadcast/mock_Watcher.go @@ -0,0 +1,163 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_broadcast + +import ( + context "context" + + messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb" + mock "github.com/stretchr/testify/mock" +) + +// MockWatcher is an autogenerated mock type for the Watcher type +type MockWatcher struct { + mock.Mock +} + +type MockWatcher_Expecter struct { + mock *mock.Mock +} + +func (_m *MockWatcher) EXPECT() *MockWatcher_Expecter { + return &MockWatcher_Expecter{mock: &_m.Mock} +} + +// Close provides a mock function with given fields: +func (_m *MockWatcher) Close() { + _m.Called() +} + +// MockWatcher_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close' +type MockWatcher_Close_Call struct { + *mock.Call +} + +// Close is a helper method to define mock.On call +func (_e *MockWatcher_Expecter) Close() *MockWatcher_Close_Call { + return &MockWatcher_Close_Call{Call: _e.mock.On("Close")} +} + +func (_c *MockWatcher_Close_Call) Run(run func()) *MockWatcher_Close_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockWatcher_Close_Call) Return() *MockWatcher_Close_Call { + _c.Call.Return() + return _c +} + +func (_c *MockWatcher_Close_Call) RunAndReturn(run func()) *MockWatcher_Close_Call { + _c.Call.Return(run) + return _c +} + +// EventChan provides a mock function with given fields: +func (_m *MockWatcher) EventChan() <-chan *messagespb.BroadcastEvent { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for EventChan") + } + + var r0 <-chan *messagespb.BroadcastEvent + if rf, ok := ret.Get(0).(func() <-chan *messagespb.BroadcastEvent); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(<-chan *messagespb.BroadcastEvent) + } + } + + return r0 +} + +// MockWatcher_EventChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'EventChan' +type MockWatcher_EventChan_Call struct { + *mock.Call +} + +// EventChan is a helper method to define mock.On call +func (_e *MockWatcher_Expecter) EventChan() *MockWatcher_EventChan_Call { + return &MockWatcher_EventChan_Call{Call: _e.mock.On("EventChan")} +} + +func (_c *MockWatcher_EventChan_Call) Run(run func()) *MockWatcher_EventChan_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockWatcher_EventChan_Call) Return(_a0 <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockWatcher_EventChan_Call) RunAndReturn(run func() <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call { + _c.Call.Return(run) + return _c +} + +// ObserveResourceKeyEvent provides a mock function with given fields: ctx, ev +func (_m *MockWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error { + ret := _m.Called(ctx, ev) + + if len(ret) == 0 { + panic("no return value specified for ObserveResourceKeyEvent") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, *messagespb.BroadcastEvent) error); ok { + r0 = rf(ctx, ev) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockWatcher_ObserveResourceKeyEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ObserveResourceKeyEvent' +type MockWatcher_ObserveResourceKeyEvent_Call struct { + *mock.Call +} + +// ObserveResourceKeyEvent is a helper method to define mock.On call +// - ctx context.Context +// - ev *messagespb.BroadcastEvent +func (_e *MockWatcher_Expecter) ObserveResourceKeyEvent(ctx interface{}, ev interface{}) *MockWatcher_ObserveResourceKeyEvent_Call { + return &MockWatcher_ObserveResourceKeyEvent_Call{Call: _e.mock.On("ObserveResourceKeyEvent", ctx, ev)} +} + +func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockWatcher_ObserveResourceKeyEvent_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent)) + }) + return _c +} + +func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Return(_a0 error) *MockWatcher_ObserveResourceKeyEvent_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockWatcher_ObserveResourceKeyEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockWatcher_ObserveResourceKeyEvent_Call { + _c.Call.Return(run) + return _c +} + +// NewMockWatcher creates a new instance of MockWatcher. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockWatcher(t interface { + mock.TestingT + Cleanup(func()) +}) *MockWatcher { + mock := &MockWatcher{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go b/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go index 3c84e0cce1f5d..850eab8ad0e4a 100644 --- a/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go +++ b/internal/mocks/streamingcoord/mock_client/mock_BroadcastService.go @@ -5,7 +5,9 @@ package mock_client import ( context "context" + messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb" message "github.com/milvus-io/milvus/pkg/streaming/util/message" + mock "github.com/stretchr/testify/mock" types "github.com/milvus-io/milvus/pkg/streaming/util/types" @@ -24,6 +26,100 @@ func (_m *MockBroadcastService) EXPECT() *MockBroadcastService_Expecter { return &MockBroadcastService_Expecter{mock: &_m.Mock} } +// Ack provides a mock function with given fields: ctx, req +func (_m *MockBroadcastService) Ack(ctx context.Context, req types.BroadcastAckRequest) error { + ret := _m.Called(ctx, req) + + if len(ret) == 0 { + panic("no return value specified for Ack") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, types.BroadcastAckRequest) error); ok { + r0 = rf(ctx, req) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockBroadcastService_Ack_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Ack' +type MockBroadcastService_Ack_Call struct { + *mock.Call +} + +// Ack is a helper method to define mock.On call +// - ctx context.Context +// - req types.BroadcastAckRequest +func (_e *MockBroadcastService_Expecter) Ack(ctx interface{}, req interface{}) *MockBroadcastService_Ack_Call { + return &MockBroadcastService_Ack_Call{Call: _e.mock.On("Ack", ctx, req)} +} + +func (_c *MockBroadcastService_Ack_Call) Run(run func(ctx context.Context, req types.BroadcastAckRequest)) *MockBroadcastService_Ack_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(types.BroadcastAckRequest)) + }) + return _c +} + +func (_c *MockBroadcastService_Ack_Call) Return(_a0 error) *MockBroadcastService_Ack_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockBroadcastService_Ack_Call) RunAndReturn(run func(context.Context, types.BroadcastAckRequest) error) *MockBroadcastService_Ack_Call { + _c.Call.Return(run) + 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) @@ -83,6 +179,38 @@ func (_c *MockBroadcastService_Broadcast_Call) RunAndReturn(run func(context.Con return _c } +// Close provides a mock function with given fields: +func (_m *MockBroadcastService) Close() { + _m.Called() +} + +// MockBroadcastService_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close' +type MockBroadcastService_Close_Call struct { + *mock.Call +} + +// Close is a helper method to define mock.On call +func (_e *MockBroadcastService_Expecter) Close() *MockBroadcastService_Close_Call { + return &MockBroadcastService_Close_Call{Call: _e.mock.On("Close")} +} + +func (_c *MockBroadcastService_Close_Call) Run(run func()) *MockBroadcastService_Close_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockBroadcastService_Close_Call) Return() *MockBroadcastService_Close_Call { + _c.Call.Return() + return _c +} + +func (_c *MockBroadcastService_Close_Call) RunAndReturn(run func()) *MockBroadcastService_Close_Call { + _c.Call.Return(run) + return _c +} + // NewMockBroadcastService creates a new instance of MockBroadcastService. 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 NewMockBroadcastService(t interface { diff --git a/internal/mocks/streamingcoord/server/mock_broadcaster/mock_Watcher.go b/internal/mocks/streamingcoord/server/mock_broadcaster/mock_Watcher.go new file mode 100644 index 0000000000000..4af7779bee8c9 --- /dev/null +++ b/internal/mocks/streamingcoord/server/mock_broadcaster/mock_Watcher.go @@ -0,0 +1,163 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_broadcaster + +import ( + context "context" + + messagespb "github.com/milvus-io/milvus/pkg/proto/messagespb" + mock "github.com/stretchr/testify/mock" +) + +// MockWatcher is an autogenerated mock type for the Watcher type +type MockWatcher struct { + mock.Mock +} + +type MockWatcher_Expecter struct { + mock *mock.Mock +} + +func (_m *MockWatcher) EXPECT() *MockWatcher_Expecter { + return &MockWatcher_Expecter{mock: &_m.Mock} +} + +// Close provides a mock function with given fields: +func (_m *MockWatcher) Close() { + _m.Called() +} + +// MockWatcher_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close' +type MockWatcher_Close_Call struct { + *mock.Call +} + +// Close is a helper method to define mock.On call +func (_e *MockWatcher_Expecter) Close() *MockWatcher_Close_Call { + return &MockWatcher_Close_Call{Call: _e.mock.On("Close")} +} + +func (_c *MockWatcher_Close_Call) Run(run func()) *MockWatcher_Close_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockWatcher_Close_Call) Return() *MockWatcher_Close_Call { + _c.Call.Return() + return _c +} + +func (_c *MockWatcher_Close_Call) RunAndReturn(run func()) *MockWatcher_Close_Call { + _c.Call.Return(run) + return _c +} + +// EventChan provides a mock function with given fields: +func (_m *MockWatcher) EventChan() <-chan *messagespb.BroadcastEvent { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for EventChan") + } + + var r0 <-chan *messagespb.BroadcastEvent + if rf, ok := ret.Get(0).(func() <-chan *messagespb.BroadcastEvent); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(<-chan *messagespb.BroadcastEvent) + } + } + + return r0 +} + +// MockWatcher_EventChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'EventChan' +type MockWatcher_EventChan_Call struct { + *mock.Call +} + +// EventChan is a helper method to define mock.On call +func (_e *MockWatcher_Expecter) EventChan() *MockWatcher_EventChan_Call { + return &MockWatcher_EventChan_Call{Call: _e.mock.On("EventChan")} +} + +func (_c *MockWatcher_EventChan_Call) Run(run func()) *MockWatcher_EventChan_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockWatcher_EventChan_Call) Return(_a0 <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockWatcher_EventChan_Call) RunAndReturn(run func() <-chan *messagespb.BroadcastEvent) *MockWatcher_EventChan_Call { + _c.Call.Return(run) + return _c +} + +// ObserveResourceKeyEvent provides a mock function with given fields: ctx, ev +func (_m *MockWatcher) ObserveResourceKeyEvent(ctx context.Context, ev *messagespb.BroadcastEvent) error { + ret := _m.Called(ctx, ev) + + if len(ret) == 0 { + panic("no return value specified for ObserveResourceKeyEvent") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, *messagespb.BroadcastEvent) error); ok { + r0 = rf(ctx, ev) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockWatcher_ObserveResourceKeyEvent_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ObserveResourceKeyEvent' +type MockWatcher_ObserveResourceKeyEvent_Call struct { + *mock.Call +} + +// ObserveResourceKeyEvent is a helper method to define mock.On call +// - ctx context.Context +// - ev *messagespb.BroadcastEvent +func (_e *MockWatcher_Expecter) ObserveResourceKeyEvent(ctx interface{}, ev interface{}) *MockWatcher_ObserveResourceKeyEvent_Call { + return &MockWatcher_ObserveResourceKeyEvent_Call{Call: _e.mock.On("ObserveResourceKeyEvent", ctx, ev)} +} + +func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Run(run func(ctx context.Context, ev *messagespb.BroadcastEvent)) *MockWatcher_ObserveResourceKeyEvent_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*messagespb.BroadcastEvent)) + }) + return _c +} + +func (_c *MockWatcher_ObserveResourceKeyEvent_Call) Return(_a0 error) *MockWatcher_ObserveResourceKeyEvent_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockWatcher_ObserveResourceKeyEvent_Call) RunAndReturn(run func(context.Context, *messagespb.BroadcastEvent) error) *MockWatcher_ObserveResourceKeyEvent_Call { + _c.Call.Return(run) + return _c +} + +// NewMockWatcher creates a new instance of MockWatcher. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockWatcher(t interface { + mock.TestingT + Cleanup(func()) +}) *MockWatcher { + mock := &MockWatcher{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index 2f39ba4265e21..6d2c6b8d26457 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -43,6 +43,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord" grpcdatacoordclient2 "github.com/milvus-io/milvus/internal/distributed/datacoord/client" grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode" @@ -366,6 +367,8 @@ func TestProxy(t *testing.T) { var wg sync.WaitGroup paramtable.Init() params := paramtable.Get() + coordclient.ResetRegistration() + params.RootCoordGrpcServerCfg.IP = "localhost" params.QueryCoordGrpcServerCfg.IP = "localhost" params.DataCoordGrpcServerCfg.IP = "localhost" diff --git a/internal/rootcoord/broadcast_task.go b/internal/rootcoord/broadcast_task.go index d95000318c3ab..9dfa06674850d 100644 --- a/internal/rootcoord/broadcast_task.go +++ b/internal/rootcoord/broadcast_task.go @@ -14,14 +14,6 @@ import ( var _ task = (*broadcastTask)(nil) -// newBroadcastTask creates a new broadcast task. -func newBroadcastTask(ctx context.Context, core *Core, msgs []message.MutableMessage) *broadcastTask { - return &broadcastTask{ - baseTask: newBaseTask(ctx, core), - msgs: msgs, - } -} - // BroadcastTask is used to implement the broadcast operation based on the msgstream // by using the streaming service interface. // msgstream will be deprecated since 2.6.0 with streaming service, so those code will be removed in the future version. diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 951b23d5806e9..fa463ecfa1863 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -31,6 +31,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/mocks" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" @@ -1358,6 +1359,7 @@ func TestRootcoord_EnableActiveStandby(t *testing.T) { randVal := rand.Int() paramtable.Init() registry.ResetRegistration() + coordclient.ResetRegistration() Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal)) // Need to reset global etcd to follow new path kvfactory.CloseEtcdClient() @@ -1419,6 +1421,7 @@ func TestRootcoord_DisableActiveStandby(t *testing.T) { randVal := rand.Int() paramtable.Init() registry.ResetRegistration() + coordclient.ResetRegistration() Params.Save("etcd.rootPath", fmt.Sprintf("/%d", randVal)) // Need to reset global etcd to follow new path kvfactory.CloseEtcdClient() diff --git a/internal/streamingcoord/client/broadcast/broadcast_impl.go b/internal/streamingcoord/client/broadcast/broadcast_impl.go index 5c5c825b3059d..7538fcd4cd276 100644 --- a/internal/streamingcoord/client/broadcast/broadcast_impl.go +++ b/internal/streamingcoord/client/broadcast/broadcast_impl.go @@ -2,30 +2,47 @@ package broadcast import ( "context" + "time" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" + "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) -// NewBroadcastService creates a new broadcast service. -func NewBroadcastService(walName string, service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient]) *BroadcastServiceImpl { - return &BroadcastServiceImpl{ +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, } } -// BroadcastServiceImpl is the implementation of BroadcastService. -type BroadcastServiceImpl struct { +// GRPCBroadcastServiceImpl is the implementation of BroadcastService based on grpc service. +// If the streaming coord is not deployed at current node, these implementation will be used. +type GRPCBroadcastServiceImpl struct { walName string service lazygrpc.Service[streamingpb.StreamingCoordBroadcastServiceClient] + w *resumingWatcher } -// Broadcast sends a broadcast message to the streaming coord to perform a broadcast. -func (c *BroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { +func (c *GRPCBroadcastServiceImpl) Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { client, err := c.service.GetService(ctx) if err != nil { return nil, err @@ -52,5 +69,28 @@ func (c *BroadcastServiceImpl) Broadcast(ctx context.Context, msg message.Broadc Extra: result.GetExtra(), } } - return &types.BroadcastAppendResult{AppendResults: results}, nil + return &types.BroadcastAppendResult{ + BroadcastID: resp.BroadcastId, + AppendResults: results, + }, nil +} + +func (c *GRPCBroadcastServiceImpl) Ack(ctx context.Context, req types.BroadcastAckRequest) error { + client, err := c.service.GetService(ctx) + if err != nil { + return err + } + _, err = client.Ack(ctx, &streamingpb.BroadcastAckRequest{ + BroadcastId: req.BroadcastID, + Vchannel: req.VChannel, + }) + 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 new file mode 100644 index 0000000000000..e335523bec869 --- /dev/null +++ b/internal/streamingcoord/client/broadcast/broadcast_test.go @@ -0,0 +1,33 @@ +package broadcast + +import ( + "context" + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" +) + +func TestBroadcast(t *testing.T) { + s := newMockServer(t, 0) + bs := NewGRPCBroadcastService(walimplstest.WALName, s) + msg, _ := message.NewDropCollectionMessageBuilderV1(). + WithHeader(&message.DropCollectionMessageHeader{}). + WithBody(&msgpb.DropCollectionRequest{}). + WithBroadcast([]string{"v1"}, message.NewCollectionNameResourceKey("r1")). + BuildBroadcast() + _, err := bs.Broadcast(context.Background(), msg) + assert.NoError(t, err) + err = bs.Ack(context.Background(), types.BroadcastAckRequest{ + VChannel: "v1", + BroadcastID: 1, + }) + assert.NoError(t, err) + err = bs.BlockUntilEvent(context.Background(), message.NewResourceKeyAckOneBroadcastEvent(message.NewCollectionNameResourceKey("r1"))) + assert.NoError(t, err) + bs.Close() +} diff --git a/internal/streamingcoord/client/broadcast/grpc_watcher.go b/internal/streamingcoord/client/broadcast/grpc_watcher.go new file mode 100644 index 0000000000000..4f98f162fbf07 --- /dev/null +++ b/internal/streamingcoord/client/broadcast/grpc_watcher.go @@ -0,0 +1,192 @@ +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/log" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" + "github.com/milvus-io/milvus/pkg/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 new file mode 100644 index 0000000000000..0d5237d98233a --- /dev/null +++ b/internal/streamingcoord/client/broadcast/grpc_watcher_test.go @@ -0,0 +1,125 @@ +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/mocks/proto/mock_streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/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 new file mode 100644 index 0000000000000..bc98f92750b92 --- /dev/null +++ b/internal/streamingcoord/client/broadcast/watcher.go @@ -0,0 +1,22 @@ +package broadcast + +import ( + "context" + + "github.com/milvus-io/milvus/pkg/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 new file mode 100644 index 0000000000000..2f99c238d1b72 --- /dev/null +++ b/internal/streamingcoord/client/broadcast/watcher_resuming.go @@ -0,0 +1,171 @@ +package broadcast + +import ( + "context" + "time" + + "github.com/cockroachdb/errors" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" + "github.com/milvus-io/milvus/pkg/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) + nextTimer := time.After(0) + 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 + } + } + case ev, ok := <-eventChan: + if !ok { + watcher.Close() + watcher = nil + break + } + r.evs.Notify(ev) + case <-nextTimer: + var err error + 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() + nextTimer = nil + } + if watcher == 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 new file mode 100644 index 0000000000000..60cbfb1e5acf8 --- /dev/null +++ b/internal/streamingcoord/client/broadcast/watcher_resuming_test.go @@ -0,0 +1,101 @@ +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/proto/messagespb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/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 4d9f30c35a95c..2537bea20b961 100644 --- a/internal/streamingcoord/client/client.go +++ b/internal/streamingcoord/client/client.go @@ -40,6 +40,15 @@ type AssignmentService interface { type BroadcastService interface { // Broadcast sends a broadcast message to the streaming service. Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) + + // 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() } // Client is the interface of log service client. @@ -84,7 +93,7 @@ func NewClient(etcdCli *clientv3.Client) Client { conn: conn, rb: rb, assignmentService: assignmentServiceImpl, - broadcastService: broadcast.NewBroadcastService(util.MustSelectWALName(), broadcastService), + broadcastService: broadcast.NewGRPCBroadcastService(util.MustSelectWALName(), broadcastService), } } diff --git a/internal/streamingcoord/client/client_impl.go b/internal/streamingcoord/client/client_impl.go index e45b6ebd2093e..dd39bd368abef 100644 --- a/internal/streamingcoord/client/client_impl.go +++ b/internal/streamingcoord/client/client_impl.go @@ -2,7 +2,6 @@ package client import ( "github.com/milvus-io/milvus/internal/streamingcoord/client/assignment" - "github.com/milvus-io/milvus/internal/streamingcoord/client/broadcast" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" ) @@ -12,7 +11,7 @@ type clientImpl struct { conn lazygrpc.Conn rb resolver.Builder assignmentService *assignment.AssignmentServiceImpl - broadcastService *broadcast.BroadcastServiceImpl + broadcastService BroadcastService } func (c *clientImpl) Broadcast() BroadcastService { @@ -29,6 +28,7 @@ func (c *clientImpl) Close() { if c.assignmentService != nil { c.assignmentService.Close() } + c.broadcastService.Close() c.conn.Close() c.rb.Close() } diff --git a/internal/streamingcoord/server/broadcaster/broadcast_manager.go b/internal/streamingcoord/server/broadcaster/broadcast_manager.go new file mode 100644 index 0000000000000..476117f1318cb --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/broadcast_manager.go @@ -0,0 +1,182 @@ +package broadcaster + +import ( + "context" + "fmt" + "sync" + + "github.com/cockroachdb/errors" + "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/log" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +// newBroadcastTaskManager creates a new broadcast task manager with recovery info. +func newBroadcastTaskManager(protos []*streamingpb.BroadcastTask) (*broadcastTaskManager, []*pendingBroadcastTask) { + logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster")) + recoveryTasks := make([]*broadcastTask, 0, len(protos)) + for _, proto := range protos { + t := newBroadcastTaskFromProto(proto) + t.SetLogger(logger.With(zap.Uint64("broadcastID", t.header.BroadcastID))) + recoveryTasks = append(recoveryTasks, t) + } + rks := make(map[message.ResourceKey]uint64, len(recoveryTasks)) + tasks := make(map[uint64]*broadcastTask, len(recoveryTasks)) + pendingTasks := make([]*pendingBroadcastTask, 0, len(recoveryTasks)) + for _, task := range recoveryTasks { + for rk := range task.header.ResourceKeys { + if oldTaskID, ok := rks[rk]; ok { + panic(fmt.Sprintf("unreachable: dirty recovery info in metastore, broadcast ids: [%d, %d]", oldTaskID, task.header.BroadcastID)) + } + rks[rk] = task.header.BroadcastID + } + tasks[task.header.BroadcastID] = task + if task.task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING { + // only the task is pending need to be reexecuted. + pendingTasks = append(pendingTasks, newPendingBroadcastTask(task)) + } + } + m := &broadcastTaskManager{ + Binder: log.Binder{}, + cond: syncutil.NewContextCond(&sync.Mutex{}), + tasks: tasks, + resourceKeys: rks, + version: 1, + } + m.SetLogger(logger) + return m, pendingTasks +} + +// broadcastTaskManager is the manager of the broadcast task. +type broadcastTaskManager struct { + log.Binder + cond *syncutil.ContextCond + tasks map[uint64]*broadcastTask // map the broadcastID to the broadcastTaskState + resourceKeys map[message.ResourceKey]uint64 // map the resource key to the broadcastID + version int // version is used to make sure that there's no update lost for watcher. +} + +// AddTask adds a new broadcast task into the manager. +func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.BroadcastMutableMessage) (*pendingBroadcastTask, error) { + id, err := resource.Resource().IDAllocator().Allocate(ctx) + if err != nil { + return nil, errors.Wrapf(err, "allocate new id failed") + } + msg = msg.WithBroadcastID(id) + + task, err := bm.addBroadcastTask(msg) + if err != nil { + return nil, err + } + return newPendingBroadcastTask(task), nil +} + +// Ack acknowledges the message at the specified vchannel. +func (bm *broadcastTaskManager) Ack(ctx context.Context, broadcastID uint64, vchannel string) error { + task, ok := bm.getBroadcastTaskByID(broadcastID) + if !ok { + bm.Logger().Warn("broadcast task not found, it may already acked, ignore the request", zap.Uint64("broadcastID", broadcastID), zap.String("vchannel", vchannel)) + return nil + } + if err := task.Ack(ctx, vchannel); err != nil { + return err + } + + 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() + defer bm.cond.L.Unlock() + + broadcastID, ok := bm.resourceKeys[resourceKey] + if !ok { + return nil, false + } + task, ok := bm.tasks[broadcastID] + return task, ok +} + +// addBroadcastTask adds the broadcast task into the manager. +func (bm *broadcastTaskManager) addBroadcastTask(msg message.BroadcastMutableMessage) (*broadcastTask, error) { + newIncomingTask := newBroadcastTaskFromBroadcastMessage(msg) + 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)) + } + } + // setup the resource keys to make resource exclusive held. + for key := range header.ResourceKeys { + bm.resourceKeys[key] = header.BroadcastID + } + bm.tasks[header.BroadcastID] = newIncomingTask + return newIncomingTask, nil +} + +// 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 +} + +// removeBroadcastTask removes the broadcast task by the broadcastID. +func (bm *broadcastTaskManager) removeBroadcastTask(broadcastID uint64) { + bm.cond.LockAndBroadcast() + defer bm.cond.L.Unlock() + + 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) + } + 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 new file mode 100644 index 0000000000000..92c322875ed2c --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/broadcast_task.go @@ -0,0 +1,198 @@ +package broadcaster + +import ( + "context" + "sync" + + "github.com/cockroachdb/errors" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +// newBroadcastTaskFromProto creates a new broadcast task from the proto. +func newBroadcastTaskFromProto(proto *streamingpb.BroadcastTask) *broadcastTask { + msg := message.NewBroadcastMutableMessageBeforeAppend(proto.Message.Payload, proto.Message.Properties) + bh := msg.BroadcastHeader() + ackedCount := 0 + for _, acked := range proto.AckedVchannelBitmap { + ackedCount += int(acked) + } + return &broadcastTask{ + mu: sync.Mutex{}, + header: bh, + task: proto, + ackedCount: ackedCount, + recoverPersisted: true, // the task is recovered from the recovery info, so it's persisted. + } +} + +// newBroadcastTaskFromBroadcastMessage creates a new broadcast task from the broadcast message. +func newBroadcastTaskFromBroadcastMessage(msg message.BroadcastMutableMessage) *broadcastTask { + header := msg.BroadcastHeader() + return &broadcastTask{ + Binder: log.Binder{}, + mu: sync.Mutex{}, + header: header, + task: &streamingpb.BroadcastTask{ + Message: &messagespb.Message{Payload: msg.Payload(), Properties: msg.Properties().ToRawMap()}, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + AckedVchannelBitmap: make([]byte, len(header.VChannels)), + }, + ackedCount: 0, + recoverPersisted: false, + } +} + +// broadcastTask is the state of the broadcast task. +type broadcastTask struct { + log.Binder + mu sync.Mutex + header *message.BroadcastHeader + task *streamingpb.BroadcastTask + ackedCount int // the count of the acked vchannels, the idompotenace is promised by task's bitmap. + // always keep same with the positive counter of task's acked_bitmap. + recoverPersisted bool // a flag to indicate that the task has been persisted into the recovery info and can be recovered. +} + +// Header returns the header of the broadcast task. +func (b *broadcastTask) Header() *message.BroadcastHeader { + // header is a immutable field, no need to lock. + return b.header +} + +// State returns the State of the broadcast task. +func (b *broadcastTask) State() streamingpb.BroadcastTaskState { + b.mu.Lock() + defer b.mu.Unlock() + + return b.task.State +} + +// PendingBroadcastMessages returns the pending broadcast message of current broad cast. +func (b *broadcastTask) PendingBroadcastMessages() []message.MutableMessage { + b.mu.Lock() + defer b.mu.Unlock() + + msg := message.NewBroadcastMutableMessageBeforeAppend(b.task.Message.Payload, b.task.Message.Properties) + msgs := msg.SplitIntoMutableMessage() + // If there's no vchannel acked, return all the messages directly. + if b.ackedCount == 0 { + return msgs + } + // filter out the vchannel that has been acked. + pendingMessages := make([]message.MutableMessage, 0, len(msgs)) + for i, msg := range msgs { + if b.task.AckedVchannelBitmap[i] != 0 { + continue + } + pendingMessages = append(pendingMessages, msg) + } + return pendingMessages +} + +// InitializeRecovery initializes the recovery of the broadcast task. +func (b *broadcastTask) InitializeRecovery(ctx context.Context) error { + b.mu.Lock() + defer b.mu.Unlock() + + if b.recoverPersisted { + return nil + } + if err := b.saveTask(ctx, b.Logger()); err != nil { + return err + } + b.recoverPersisted = true + return nil +} + +// Ack acknowledges the message at the specified vchannel. +func (b *broadcastTask) Ack(ctx context.Context, vchannel string) error { + b.mu.Lock() + defer b.mu.Unlock() + + b.setVChannelAcked(vchannel) + if b.isAllDone() { + // All vchannels are acked, mark the task as done, even if there are still pending messages on working. + // The pending messages is repeated sent operation, can be ignored. + b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE + } + // We should always save the task after acked. + // Even if the task mark as done in memory. + // Because the task is set as done in memory before save the recovery info. + return b.saveTask(ctx, b.Logger().With(zap.String("ackVChannel", vchannel))) +} + +// setVChannelAcked sets the vchannel as acked. +func (b *broadcastTask) setVChannelAcked(vchannel string) { + idx, err := b.findIdxOfVChannel(vchannel) + if err != nil { + panic(err) + } + b.task.AckedVchannelBitmap[idx] = 1 + // Check if all vchannels are acked. + ackedCount := 0 + for _, acked := range b.task.AckedVchannelBitmap { + ackedCount += int(acked) + } + b.ackedCount = ackedCount +} + +// findIdxOfVChannel finds the index of the vchannel in the broadcast task. +func (b *broadcastTask) findIdxOfVChannel(vchannel string) (int, error) { + for i, channelName := range b.header.VChannels { + if channelName == vchannel { + return i, nil + } + } + return -1, errors.Errorf("unreachable: vchannel is %s not found in the broadcast task", vchannel) +} + +// isAllDone check if all the vchannels are acked. +func (b *broadcastTask) isAllDone() bool { + return b.ackedCount == len(b.header.VChannels) +} + +// BroadcastDone marks the broadcast operation is done. +func (b *broadcastTask) BroadcastDone(ctx context.Context) error { + b.mu.Lock() + defer b.mu.Unlock() + + if b.isAllDone() { + // If all vchannels are acked, mark the task as done. + b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE + } else { + // There's no more pending message, mark the task as wait ack. + b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_WAIT_ACK + } + return b.saveTask(ctx, b.Logger()) +} + +// IsAllAcked returns true if all the vchannels are acked. +func (b *broadcastTask) IsAllAcked() bool { + b.mu.Lock() + defer b.mu.Unlock() + return b.isAllDone() +} + +// IsAcked returns true if any vchannel is acked. +func (b *broadcastTask) IsAcked() bool { + b.mu.Lock() + defer b.mu.Unlock() + return b.ackedCount > 0 +} + +// saveTask saves the broadcast task recovery info. +func (b *broadcastTask) saveTask(ctx context.Context, logger *log.MLogger) error { + logger = logger.With(zap.String("state", b.task.State.String()), zap.Int("ackedVChannelCount", b.ackedCount)) + if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, b.header.BroadcastID, b.task); err != nil { + logger.Warn("save broadcast task failed", zap.Error(err)) + return err + } + logger.Info("save broadcast task done") + return nil +} diff --git a/internal/streamingcoord/server/broadcaster/broadcaster.go b/internal/streamingcoord/server/broadcaster/broadcaster.go index b1b6b3c633fc4..8a5a333375117 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster.go @@ -12,8 +12,26 @@ type Broadcaster interface { // Broadcast broadcasts the message to all channels. Broadcast(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) + // 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 0b63ba32880b8..7bedcd3eceeb8 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_impl.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_impl.go @@ -10,8 +10,6 @@ import ( "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/proto/messagespb" - "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/contextutil" @@ -25,28 +23,20 @@ func RecoverBroadcaster( ctx context.Context, appendOperator *syncutil.Future[AppendOperator], ) (Broadcaster, error) { - logger := resource.Resource().Logger().With(log.FieldComponent("broadcaster")) tasks, err := resource.Resource().StreamingCatalog().ListBroadcastTask(ctx) if err != nil { return nil, err } - pendings := make([]*broadcastTask, 0, len(tasks)) - for _, task := range tasks { - if task.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING { - // recover pending task - t := newTask(task, logger) - pendings = append(pendings, t) - } - } + manager, pendings := newBroadcastTaskManager(tasks) b := &broadcasterImpl{ - logger: logger, + manager: manager, lifetime: typeutil.NewLifetime(), backgroundTaskNotifier: syncutil.NewAsyncTaskNotifier[struct{}](), pendings: pendings, - backoffs: typeutil.NewHeap[*broadcastTask](&broadcastTaskArray{}), - backoffChan: make(chan *broadcastTask), - pendingChan: make(chan *broadcastTask), - workerChan: make(chan *broadcastTask), + backoffs: typeutil.NewHeap[*pendingBroadcastTask](&pendingBroadcastTaskArray{}), + backoffChan: make(chan *pendingBroadcastTask), + pendingChan: make(chan *pendingBroadcastTask), + workerChan: make(chan *pendingBroadcastTask), appendOperator: appendOperator, } go b.execute() @@ -55,14 +45,14 @@ func RecoverBroadcaster( // broadcasterImpl is the implementation of Broadcaster type broadcasterImpl struct { - logger *log.MLogger + manager *broadcastTaskManager lifetime *typeutil.Lifetime backgroundTaskNotifier *syncutil.AsyncTaskNotifier[struct{}] - pendings []*broadcastTask - backoffs typeutil.Heap[*broadcastTask] - pendingChan chan *broadcastTask - backoffChan chan *broadcastTask - workerChan chan *broadcastTask + pendings []*pendingBroadcastTask + backoffs typeutil.Heap[*pendingBroadcastTask] + pendingChan chan *pendingBroadcastTask + backoffChan chan *pendingBroadcastTask + workerChan chan *pendingBroadcastTask appendOperator *syncutil.Future[AppendOperator] // TODO: we can remove those lazy future in 2.6.0, by remove the msgstream broadcaster. } @@ -72,18 +62,17 @@ func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMu return nil, status.NewOnShutdownError("broadcaster is closing") } defer func() { + b.lifetime.Done() if err != nil { - b.logger.Warn("broadcast message failed", zap.Error(err)) + b.Logger().Warn("broadcast message failed", zap.Error(err)) return } }() - // Once the task is persisted, it must be successful. - task, err := b.persistBroadcastTask(ctx, msg) + t, err := b.manager.AddTask(ctx, msg) if err != nil { return nil, err } - t := newTask(task, b.logger) select { case <-b.backgroundTaskNotifier.Context().Done(): // We can only check the background context but not the request context here. @@ -98,24 +87,23 @@ func (b *broadcasterImpl) Broadcast(ctx context.Context, msg message.BroadcastMu return t.BlockUntilTaskDone(ctx) } -// persistBroadcastTask persists the broadcast task into catalog. -func (b *broadcasterImpl) persistBroadcastTask(ctx context.Context, msg message.BroadcastMutableMessage) (*streamingpb.BroadcastTask, error) { +// Ack acknowledges the message at the specified vchannel. +func (b *broadcasterImpl) Ack(ctx context.Context, req types.BroadcastAckRequest) error { + if !b.lifetime.Add(typeutil.LifetimeStateWorking) { + return status.NewOnShutdownError("broadcaster is closing") + } defer b.lifetime.Done() - id, err := resource.Resource().IDAllocator().Allocate(ctx) - if err != nil { - return nil, status.NewInner("allocate new id failed, %s", err.Error()) - } - task := &streamingpb.BroadcastTask{ - TaskId: int64(id), - Message: &messagespb.Message{Payload: msg.Payload(), Properties: msg.Properties().ToRawMap()}, - State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, - } - // Save the task into catalog to help recovery. - if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, task); err != nil { - return nil, status.NewInner("save broadcast task failed, %s", err.Error()) + 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") } - return task, nil + defer b.lifetime.Done() + + return newWatcher(b), nil } func (b *broadcasterImpl) Close() { @@ -126,26 +114,30 @@ func (b *broadcasterImpl) Close() { b.backgroundTaskNotifier.BlockUntilFinish() } +func (b *broadcasterImpl) Logger() *log.MLogger { + return b.manager.Logger() +} + // execute the broadcaster func (b *broadcasterImpl) execute() { workers := int(float64(hardware.GetCPUNum()) * paramtable.Get().StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat()) if workers < 1 { workers = 1 } - b.logger.Info("broadcaster start to execute", zap.Int("workerNum", workers)) + b.Logger().Info("broadcaster start to execute", zap.Int("workerNum", workers)) defer func() { b.backgroundTaskNotifier.Finish(struct{}{}) - b.logger.Info("broadcaster execute exit") + b.Logger().Info("broadcaster execute exit") }() // Wait for appendOperator ready appendOperator, err := b.appendOperator.GetWithContext(b.backgroundTaskNotifier.Context()) if err != nil { - b.logger.Info("broadcaster is closed before appendOperator ready") + b.Logger().Info("broadcaster is closed before appendOperator ready") return } - b.logger.Info("broadcaster appendOperator ready, begin to start workers and dispatch") + b.Logger().Info("broadcaster appendOperator ready, begin to start workers and dispatch") // Start n workers to handle the broadcast task. wg := sync.WaitGroup{} @@ -165,8 +157,8 @@ func (b *broadcasterImpl) execute() { func (b *broadcasterImpl) dispatch() { for { - var workerChan chan *broadcastTask - var nextTask *broadcastTask + var workerChan chan *pendingBroadcastTask + var nextTask *pendingBroadcastTask var nextBackOff <-chan time.Time // Wait for new task. if len(b.pendings) > 0 { @@ -176,7 +168,7 @@ func (b *broadcasterImpl) dispatch() { if b.backoffs.Len() > 0 { var nextInterval time.Duration nextBackOff, nextInterval = b.backoffs.Peek().NextTimer() - b.logger.Info("backoff task", zap.Duration("nextInterval", nextInterval)) + b.Logger().Info("backoff task", zap.Duration("nextInterval", nextInterval)) } select { @@ -189,7 +181,7 @@ func (b *broadcasterImpl) dispatch() { b.backoffs.Push(task) case <-nextBackOff: // backoff is done, move all the backoff done task into pending to retry. - newPops := make([]*broadcastTask, 0) + newPops := make([]*pendingBroadcastTask, 0) for b.backoffs.Len() > 0 && b.backoffs.Peek().NextInterval() < time.Millisecond { newPops = append(newPops, b.backoffs.Pop()) } @@ -205,7 +197,7 @@ func (b *broadcasterImpl) dispatch() { } func (b *broadcasterImpl) worker(no int, appendOperator AppendOperator) { - logger := b.logger.With(zap.Int("workerNo", no)) + logger := b.Logger().With(zap.Int("workerNo", no)) defer func() { logger.Info("broadcaster worker exit") }() diff --git a/internal/streamingcoord/server/broadcaster/broadcaster_test.go b/internal/streamingcoord/server/broadcaster/broadcaster_test.go index 738057dfa30cb..e66c8ffd2498e 100644 --- a/internal/streamingcoord/server/broadcaster/broadcaster_test.go +++ b/internal/streamingcoord/server/broadcaster/broadcaster_test.go @@ -3,6 +3,7 @@ package broadcaster import ( "context" "math/rand" + "sync" "testing" "time" @@ -10,6 +11,7 @@ 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" @@ -17,6 +19,7 @@ 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/proto/messagespb" "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" @@ -24,6 +27,7 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) func TestBroadcaster(t *testing.T) { @@ -33,19 +37,37 @@ 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"}), - createNewBroadcastTask(2, []string{"v1", "v2"}), + createNewBroadcastTask(1, []string{"v1"}, message.NewCollectionNameResourceKey("c1")), + createNewBroadcastTask(2, []string{"v1", "v2"}, message.NewCollectionNameResourceKey("c2")), createNewBroadcastTask(3, []string{"v1", "v2", "v3"}), + createNewWaitAckBroadcastTaskFromMessage( + createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(4), + streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + []byte{0x00, 0x01, 0x00}), + createNewWaitAckBroadcastTaskFromMessage( + createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(5), + streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + []byte{0x01, 0x01, 0x00}), + createNewWaitAckBroadcastTaskFromMessage( + createNewBroadcastMsg([]string{"v1", "v2", "v3"}).WithBroadcastID(6), // will be done directly. + streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + []byte{0x01, 0x01, 0x01}), + createNewWaitAckBroadcastTaskFromMessage( + createNewBroadcastMsg([]string{"v1", "v2", "v3"}, + message.NewCollectionNameResourceKey("c3"), + message.NewCollectionNameResourceKey("c4")).WithBroadcastID(7), + streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_WAIT_ACK, + []byte{0x00, 0x00, 0x00}), }, nil }).Times(1) - done := atomic.NewInt64(0) - meta.EXPECT().SaveBroadcastTask(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, bt *streamingpb.BroadcastTask) error { + done := typeutil.NewConcurrentSet[uint64]() + meta.EXPECT().SaveBroadcastTask(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, broadcastID uint64, bt *streamingpb.BroadcastTask) error { // may failure if rand.Int31n(10) < 3 { return errors.New("save task failed") } if bt.State == streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE { - done.Inc() + done.Insert(broadcastID) } return nil }) @@ -59,30 +81,145 @@ func TestBroadcaster(t *testing.T) { assert.NoError(t, err) assert.NotNil(t, bc) assert.Eventually(t, func() bool { - return appended.Load() == 6 && done.Load() == 3 + return appended.Load() == 9 && len(done.Collect()) == 1 // 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() + + assert.Eventually(t, func() bool { + return len(done.Collect()) == 2 + }, 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"})) + result, err = bc.Broadcast(context.Background(), createNewBroadcastMsg([]string{"v1", "v2", "v3"}, message.NewCollectionNameResourceKey("c7"))) if err == nil { break } } - assert.Equal(t, int(appended.Load()), 9) + assert.Equal(t, int(appended.Load()), 12) assert.Equal(t, len(result.AppendResults), 3) - assert.Eventually(t, func() bool { - return done.Load() == 4 + return len(done.Collect()) == 2 }, 30*time.Second, 10*time.Millisecond) - // TODO: error path. + // 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"})) assert.Error(t, err) assert.Nil(t, result) + err = bc.Ack(context.Background(), types.BroadcastAckRequest{BroadcastID: 3, VChannel: "v1"}) + 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) { + for { + if err := bc.Ack(context.Background(), types.BroadcastAckRequest{ + BroadcastID: broadcastID, + VChannel: vchannel, + }); err == nil { + break + } + } +} + +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) { @@ -122,11 +259,11 @@ func createOpeartor(t *testing.T) (*syncutil.Future[AppendOperator], *atomic.Int return fOperator, appended } -func createNewBroadcastMsg(vchannels []string) message.BroadcastMutableMessage { +func createNewBroadcastMsg(vchannels []string, rks ...message.ResourceKey) message.BroadcastMutableMessage { msg, err := message.NewDropCollectionMessageBuilderV1(). WithHeader(&messagespb.DropCollectionMessageHeader{}). WithBody(&msgpb.DropCollectionRequest{}). - WithBroadcast(vchannels). + WithBroadcast(vchannels, rks...). BuildBroadcast() if err != nil { panic(err) @@ -134,14 +271,29 @@ func createNewBroadcastMsg(vchannels []string) message.BroadcastMutableMessage { return msg } -func createNewBroadcastTask(taskID int64, vchannels []string) *streamingpb.BroadcastTask { - msg := createNewBroadcastMsg(vchannels) +func createNewBroadcastTask(broadcastID uint64, vchannels []string, rks ...message.ResourceKey) *streamingpb.BroadcastTask { + msg := createNewBroadcastMsg(vchannels, rks...).WithBroadcastID(broadcastID) + return &streamingpb.BroadcastTask{ + Message: &messagespb.Message{ + Payload: msg.Payload(), + Properties: msg.Properties().ToRawMap(), + }, + State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + AckedVchannelBitmap: make([]byte, len(vchannels)), + } +} + +func createNewWaitAckBroadcastTaskFromMessage( + msg message.BroadcastMutableMessage, + state streamingpb.BroadcastTaskState, + bitmap []byte, +) *streamingpb.BroadcastTask { return &streamingpb.BroadcastTask{ - TaskId: taskID, Message: &messagespb.Message{ Payload: msg.Payload(), Properties: msg.Properties().ToRawMap(), }, - State: streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_PENDING, + State: state, + AckedVchannelBitmap: bitmap, } } diff --git a/internal/streamingcoord/server/broadcaster/task.go b/internal/streamingcoord/server/broadcaster/task.go index 38693bf509515..f74eb94ff0e38 100644 --- a/internal/streamingcoord/server/broadcaster/task.go +++ b/internal/streamingcoord/server/broadcaster/task.go @@ -7,9 +7,6 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/streamingcoord/server/resource" - "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" @@ -18,13 +15,11 @@ import ( var errBroadcastTaskIsNotDone = errors.New("broadcast task is not done") -// newTask creates a new task -func newTask(task *streamingpb.BroadcastTask, logger *log.MLogger) *broadcastTask { - bt := message.NewBroadcastMutableMessage(task.Message.Payload, task.Message.Properties) - msgs := bt.SplitIntoMutableMessage() - return &broadcastTask{ - logger: logger.With(zap.Int64("taskID", task.TaskId), zap.Int("broadcastTotal", len(msgs))), - task: task, +// newPendingBroadcastTask creates a new pendingBroadcastTask. +func newPendingBroadcastTask(task *broadcastTask) *pendingBroadcastTask { + msgs := task.PendingBroadcastMessages() + return &pendingBroadcastTask{ + broadcastTask: task, pendingMessages: msgs, appendResult: make(map[string]*types.AppendResult, len(msgs)), future: syncutil.NewFuture[*types.BroadcastAppendResult](), @@ -39,10 +34,9 @@ func newTask(task *streamingpb.BroadcastTask, logger *log.MLogger) *broadcastTas } } -// broadcastTask is the task for broadcasting messages. -type broadcastTask struct { - logger *log.MLogger - task *streamingpb.BroadcastTask +// pendingBroadcastTask is a task that is pending to be broadcasted. +type pendingBroadcastTask struct { + *broadcastTask pendingMessages []message.MutableMessage appendResult map[string]*types.AppendResult future *syncutil.Future[*types.BroadcastAppendResult] @@ -52,14 +46,20 @@ type broadcastTask struct { // Execute reexecute the task, return nil if the task is done, otherwise not done. // Execute can be repeated called until the task is done. // Same semantics as the `Poll` operation in eventloop. -func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) error { +func (b *pendingBroadcastTask) Execute(ctx context.Context, operator AppendOperator) error { + if err := b.broadcastTask.InitializeRecovery(ctx); err != nil { + b.Logger().Warn("broadcast task initialize recovery failed", zap.Error(err)) + b.UpdateInstantWithNextBackOff() + return err + } + if len(b.pendingMessages) > 0 { - b.logger.Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages))) + b.Logger().Debug("broadcast task is polling to make sent...", zap.Int("pendingMessages", len(b.pendingMessages))) resps := operator.AppendMessages(ctx, b.pendingMessages...) newPendings := make([]message.MutableMessage, 0) for idx, resp := range resps.Responses { if resp.Error != nil { - b.logger.Warn("broadcast task append message failed", zap.Int("idx", idx), zap.Error(resp.Error)) + b.Logger().Warn("broadcast task append message failed", zap.Int("idx", idx), zap.Error(resp.Error)) newPendings = append(newPendings, b.pendingMessages[idx]) continue } @@ -67,15 +67,15 @@ func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) er } b.pendingMessages = newPendings if len(newPendings) == 0 { - b.future.Set(&types.BroadcastAppendResult{AppendResults: b.appendResult}) + b.future.Set(&types.BroadcastAppendResult{ + BroadcastID: b.header.BroadcastID, + AppendResults: b.appendResult, + }) } - b.logger.Info("broadcast task make a new broadcast done", zap.Int("backoffRetryMessages", len(b.pendingMessages))) + b.Logger().Info("broadcast task make a new broadcast done", zap.Int("backoffRetryMessages", len(b.pendingMessages))) } if len(b.pendingMessages) == 0 { - // There's no more pending message, mark the task as done. - b.task.State = streamingpb.BroadcastTaskState_BROADCAST_TASK_STATE_DONE - if err := resource.Resource().StreamingCatalog().SaveBroadcastTask(ctx, b.task); err != nil { - b.logger.Warn("save broadcast task failed", zap.Error(err)) + if err := b.broadcastTask.BroadcastDone(ctx); err != nil { b.UpdateInstantWithNextBackOff() return err } @@ -86,34 +86,35 @@ func (b *broadcastTask) Execute(ctx context.Context, operator AppendOperator) er } // BlockUntilTaskDone blocks until the task is done. -func (b *broadcastTask) BlockUntilTaskDone(ctx context.Context) (*types.BroadcastAppendResult, error) { +func (b *pendingBroadcastTask) BlockUntilTaskDone(ctx context.Context) (*types.BroadcastAppendResult, error) { return b.future.GetWithContext(ctx) } -type broadcastTaskArray []*broadcastTask +// pendingBroadcastTaskArray is a heap of pendingBroadcastTask. +type pendingBroadcastTaskArray []*pendingBroadcastTask // Len returns the length of the heap. -func (h broadcastTaskArray) Len() int { +func (h pendingBroadcastTaskArray) Len() int { return len(h) } // Less returns true if the element at index i is less than the element at index j. -func (h broadcastTaskArray) Less(i, j int) bool { +func (h pendingBroadcastTaskArray) Less(i, j int) bool { return h[i].NextInstant().Before(h[j].NextInstant()) } // Swap swaps the elements at indexes i and j. -func (h broadcastTaskArray) Swap(i, j int) { h[i], h[j] = h[j], h[i] } +func (h pendingBroadcastTaskArray) Swap(i, j int) { h[i], h[j] = h[j], h[i] } // Push pushes the last one at len. -func (h *broadcastTaskArray) Push(x interface{}) { +func (h *pendingBroadcastTaskArray) Push(x interface{}) { // Push and Pop use pointer receivers because they modify the slice's length, // not just its contents. - *h = append(*h, x.(*broadcastTask)) + *h = append(*h, x.(*pendingBroadcastTask)) } // Pop pop the last one at len. -func (h *broadcastTaskArray) Pop() interface{} { +func (h *pendingBroadcastTaskArray) Pop() interface{} { old := *h n := len(old) x := old[n-1] @@ -123,6 +124,6 @@ func (h *broadcastTaskArray) Pop() interface{} { // Peek returns the element at the top of the heap. // Panics if the heap is empty. -func (h *broadcastTaskArray) Peek() interface{} { +func (h *pendingBroadcastTaskArray) Peek() interface{} { return (*h)[0] } diff --git a/internal/streamingcoord/server/broadcaster/watcher.go b/internal/streamingcoord/server/broadcaster/watcher.go new file mode 100644 index 0000000000000..e5c9e1e62fb83 --- /dev/null +++ b/internal/streamingcoord/server/broadcaster/watcher.go @@ -0,0 +1,103 @@ +package broadcaster + +import ( + "context" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/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 2bdcf7296fae8..e903d92dba9cd 100644 --- a/internal/streamingcoord/server/service/broadcast.go +++ b/internal/streamingcoord/server/service/broadcast.go @@ -4,8 +4,10 @@ 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/proto/streamingpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" ) @@ -32,7 +34,7 @@ func (s *broadcastServceImpl) Broadcast(ctx context.Context, req *streamingpb.Br if err != nil { return nil, err } - results, err := broadcaster.Broadcast(ctx, message.NewBroadcastMutableMessage(req.Message.Payload, req.Message.Properties)) + results, err := broadcaster.Broadcast(ctx, message.NewBroadcastMutableMessageBeforeAppend(req.Message.Payload, req.Message.Properties)) if err != nil { return nil, err } @@ -40,5 +42,38 @@ func (s *broadcastServceImpl) Broadcast(ctx context.Context, req *streamingpb.Br for vchannel, result := range results.AppendResults { protoResult[vchannel] = result.IntoProto() } - return &streamingpb.BroadcastResponse{Results: protoResult}, nil + return &streamingpb.BroadcastResponse{ + BroadcastId: results.BroadcastID, + Results: protoResult, + }, nil +} + +// Ack acknowledges the message at the specified vchannel. +func (s *broadcastServceImpl) Ack(ctx context.Context, req *streamingpb.BroadcastAckRequest) (*streamingpb.BroadcastAckResponse, error) { + broadcaster, err := s.broadcaster.GetWithContext(ctx) + if err != nil { + return nil, err + } + if err := broadcaster.Ack(ctx, types.BroadcastAckRequest{ + BroadcastID: req.BroadcastId, + VChannel: req.Vchannel, + }); err != nil { + return nil, err + } + 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 new file mode 100644 index 0000000000000..7240f4d2c9da8 --- /dev/null +++ b/internal/streamingcoord/server/service/broadcast/broadcast_watch_grpc_server_helper.go @@ -0,0 +1,30 @@ +package broadcast + +import ( + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/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 new file mode 100644 index 0000000000000..0c6d7a28ab93d --- /dev/null +++ b/internal/streamingcoord/server/service/broadcast/broadcast_watch_server.go @@ -0,0 +1,117 @@ +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/log" + "github.com/milvus-io/milvus/pkg/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 new file mode 100644 index 0000000000000..f68762bef3120 --- /dev/null +++ b/internal/streamingcoord/server/service/broadcast/broadcast_watch_test.go @@ -0,0 +1,76 @@ +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/mocks/proto/mock_streamingpb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/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/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go b/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go index 11129a7a9c5ce..1f323ce941fc5 100644 --- a/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go +++ b/internal/streamingcoord/server/service/discover/discover_grpc_server_helper.go @@ -45,6 +45,8 @@ func (h *discoverGrpcServerHelper) SendFullAssignment(v typeutil.VersionInt64Pai // SendCloseResponse sends the close response to client. func (h *discoverGrpcServerHelper) SendCloseResponse() error { return h.Send(&streamingpb.AssignmentDiscoverResponse{ - Response: &streamingpb.AssignmentDiscoverResponse_Close{}, + Response: &streamingpb.AssignmentDiscoverResponse_Close{ + Close: &streamingpb.CloseAssignmentDiscoverResponse{}, + }, }) } diff --git a/internal/streamingnode/client/handler/producer/producer_impl.go b/internal/streamingnode/client/handler/producer/producer_impl.go index 9c84df18429c0..8c27fbd3257c2 100644 --- a/internal/streamingnode/client/handler/producer/producer_impl.go +++ b/internal/streamingnode/client/handler/producer/producer_impl.go @@ -230,10 +230,10 @@ func (p *producerImpl) sendLoop() (err error) { } else { p.logger.Info("send arm of stream closed") } - close(p.sendExitCh) if err := p.grpcStreamClient.CloseSend(); err != nil { p.logger.Warn("failed to close send", zap.Error(err)) } + close(p.sendExitCh) }() for { @@ -265,16 +265,15 @@ func (p *producerImpl) recvLoop() (err error) { defer func() { if err != nil { p.logger.Warn("recv arm of stream closed by unexpected error", zap.Error(err)) - return + } else { + p.logger.Info("recv arm of stream closed") } - p.logger.Info("recv arm of stream closed") close(p.recvExitCh) }() for { resp, err := p.grpcStreamClient.Recv() if errors.Is(err, io.EOF) { - p.logger.Debug("stream closed successful") return nil } if err != nil { diff --git a/internal/streamingnode/server/service/handler/producer/produce_server.go b/internal/streamingnode/server/service/handler/producer/produce_server.go index a14a5778b268f..5e0b862d271b6 100644 --- a/internal/streamingnode/server/service/handler/producer/produce_server.go +++ b/internal/streamingnode/server/service/handler/producer/produce_server.go @@ -180,7 +180,7 @@ func (p *ProduceServer) handleProduce(req *streamingpb.ProduceMessageRequest) { p.appendWG.Add(1) p.logger.Debug("recv produce message from client", zap.Int64("requestID", req.RequestId)) // Update metrics. - msg := message.NewMutableMessage(req.GetMessage().GetPayload(), req.GetMessage().GetProperties()) + msg := message.NewMutableMessageBeforeAppend(req.GetMessage().GetPayload(), req.GetMessage().GetProperties()) metricsGuard := p.metrics.StartProduce() if err := p.validateMessage(msg); err != nil { p.logger.Warn("produce message validation failed", zap.Int64("requestID", req.RequestId), zap.Error(err)) diff --git a/internal/util/streamingutil/status/streaming_error.go b/internal/util/streamingutil/status/streaming_error.go index 08b2263e3478d..5adb08db5c540 100644 --- a/internal/util/streamingutil/status/streaming_error.go +++ b/internal/util/streamingutil/status/streaming_error.go @@ -55,6 +55,11 @@ func (e *StreamingError) IsTxnUnavilable() bool { e.Code == streamingpb.StreamingCode_STREAMING_CODE_INVALID_TRANSACTION_STATE } +// IsResourceAcquired returns true if the resource is acquired. +func (e *StreamingError) IsResourceAcquired() bool { + return e.Code == streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED +} + // NewOnShutdownError creates a new StreamingError with code STREAMING_CODE_ON_SHUTDOWN. func NewOnShutdownError(format string, args ...interface{}) *StreamingError { return New(streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, format, args...) @@ -116,6 +121,11 @@ func NewUnrecoverableError(format string, args ...interface{}) *StreamingError { return New(streamingpb.StreamingCode_STREAMING_CODE_UNRECOVERABLE, format, args...) } +// NewResourceAcquired creates a new StreamingError with code STREAMING_CODE_RESOURCE_ACQUIRED. +func NewResourceAcquired(format string, args ...interface{}) *StreamingError { + return New(streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED, format, args...) +} + // New creates a new StreamingError with the given code and cause. func New(code streamingpb.StreamingCode, format string, args ...interface{}) *StreamingError { if len(args) == 0 { diff --git a/internal/util/streamingutil/status/streaming_error_test.go b/internal/util/streamingutil/status/streaming_error_test.go index 04ea16931349c..3b6972a57239e 100644 --- a/internal/util/streamingutil/status/streaming_error_test.go +++ b/internal/util/streamingutil/status/streaming_error_test.go @@ -56,4 +56,10 @@ func TestStreamingError(t *testing.T) { assert.False(t, streamingErr.IsWrongStreamingNode()) pbErr = streamingErr.AsPBError() assert.Equal(t, streamingpb.StreamingCode_STREAMING_CODE_ON_SHUTDOWN, pbErr.Code) + + streamingErr = NewResourceAcquired("test, %d", 1) + assert.Contains(t, streamingErr.Error(), "code: STREAMING_CODE_RESOURCE_ACQUIRED, cause: test, 1") + assert.False(t, streamingErr.IsWrongStreamingNode()) + pbErr = streamingErr.AsPBError() + assert.Equal(t, streamingpb.StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED, pbErr.Code) } diff --git a/pkg/.mockery_pkg.yaml b/pkg/.mockery_pkg.yaml index 124f721fa14da..1a2b54eeb379c 100644 --- a/pkg/.mockery_pkg.yaml +++ b/pkg/.mockery_pkg.yaml @@ -39,3 +39,6 @@ packages: StreamingNodeHandlerServiceClient: StreamingNodeHandlerService_ConsumeClient: StreamingNodeHandlerService_ProduceClient: + StreamingCoordBroadcastService_WatchServer: + StreamingCoordBroadcastServiceClient: + StreamingCoordBroadcastService_WatchClient: diff --git a/pkg/log/with_logger.go b/pkg/log/with_logger.go new file mode 100644 index 0000000000000..8bf0a44b7a518 --- /dev/null +++ b/pkg/log/with_logger.go @@ -0,0 +1,37 @@ +package log + +var ( + _ WithLogger = &Binder{} + _ LoggerBinder = &Binder{} +) + +// WithLogger is an interface to help access local logger. +type WithLogger interface { + Logger() *MLogger +} + +// LoggerBinder is an interface to help set logger. +type LoggerBinder interface { + SetLogger(logger *MLogger) +} + +// Binder is a embedding type to help access local logger. +type Binder struct { + logger *MLogger +} + +// SetLogger sets logger to Binder. +func (w *Binder) SetLogger(logger *MLogger) { + if w.logger != nil { + panic("logger already set") + } + w.logger = logger +} + +// Logger returns the logger of Binder. +func (w *Binder) Logger() *MLogger { + if w.logger == nil { + return With() + } + return w.logger +} diff --git a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go new file mode 100644 index 0000000000000..d53b5bcca9643 --- /dev/null +++ b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastServiceClient.go @@ -0,0 +1,261 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_streamingpb + +import ( + context "context" + + grpc "google.golang.org/grpc" + + mock "github.com/stretchr/testify/mock" + + streamingpb "github.com/milvus-io/milvus/pkg/proto/streamingpb" +) + +// MockStreamingCoordBroadcastServiceClient is an autogenerated mock type for the StreamingCoordBroadcastServiceClient type +type MockStreamingCoordBroadcastServiceClient struct { + mock.Mock +} + +type MockStreamingCoordBroadcastServiceClient_Expecter struct { + mock *mock.Mock +} + +func (_m *MockStreamingCoordBroadcastServiceClient) EXPECT() *MockStreamingCoordBroadcastServiceClient_Expecter { + return &MockStreamingCoordBroadcastServiceClient_Expecter{mock: &_m.Mock} +} + +// Ack provides a mock function with given fields: ctx, in, opts +func (_m *MockStreamingCoordBroadcastServiceClient) Ack(ctx context.Context, in *streamingpb.BroadcastAckRequest, opts ...grpc.CallOption) (*streamingpb.BroadcastAckResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for Ack") + } + + var r0 *streamingpb.BroadcastAckResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) (*streamingpb.BroadcastAckResponse, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) *streamingpb.BroadcastAckResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*streamingpb.BroadcastAckResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockStreamingCoordBroadcastServiceClient_Ack_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Ack' +type MockStreamingCoordBroadcastServiceClient_Ack_Call struct { + *mock.Call +} + +// Ack is a helper method to define mock.On call +// - ctx context.Context +// - in *streamingpb.BroadcastAckRequest +// - opts ...grpc.CallOption +func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Ack(ctx interface{}, in interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Ack_Call { + return &MockStreamingCoordBroadcastServiceClient_Ack_Call{Call: _e.mock.On("Ack", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockStreamingCoordBroadcastServiceClient_Ack_Call) Run(run func(ctx context.Context, in *streamingpb.BroadcastAckRequest, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Ack_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*streamingpb.BroadcastAckRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockStreamingCoordBroadcastServiceClient_Ack_Call) Return(_a0 *streamingpb.BroadcastAckResponse, _a1 error) *MockStreamingCoordBroadcastServiceClient_Ack_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockStreamingCoordBroadcastServiceClient_Ack_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastAckRequest, ...grpc.CallOption) (*streamingpb.BroadcastAckResponse, error)) *MockStreamingCoordBroadcastServiceClient_Ack_Call { + _c.Call.Return(run) + return _c +} + +// Broadcast provides a mock function with given fields: ctx, in, opts +func (_m *MockStreamingCoordBroadcastServiceClient) Broadcast(ctx context.Context, in *streamingpb.BroadcastRequest, opts ...grpc.CallOption) (*streamingpb.BroadcastResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for Broadcast") + } + + var r0 *streamingpb.BroadcastResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) (*streamingpb.BroadcastResponse, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) *streamingpb.BroadcastResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*streamingpb.BroadcastResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockStreamingCoordBroadcastServiceClient_Broadcast_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Broadcast' +type MockStreamingCoordBroadcastServiceClient_Broadcast_Call struct { + *mock.Call +} + +// Broadcast is a helper method to define mock.On call +// - ctx context.Context +// - in *streamingpb.BroadcastRequest +// - opts ...grpc.CallOption +func (_e *MockStreamingCoordBroadcastServiceClient_Expecter) Broadcast(ctx interface{}, in interface{}, opts ...interface{}) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call { + return &MockStreamingCoordBroadcastServiceClient_Broadcast_Call{Call: _e.mock.On("Broadcast", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) Run(run func(ctx context.Context, in *streamingpb.BroadcastRequest, opts ...grpc.CallOption)) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*streamingpb.BroadcastRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) Return(_a0 *streamingpb.BroadcastResponse, _a1 error) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockStreamingCoordBroadcastServiceClient_Broadcast_Call) RunAndReturn(run func(context.Context, *streamingpb.BroadcastRequest, ...grpc.CallOption) (*streamingpb.BroadcastResponse, error)) *MockStreamingCoordBroadcastServiceClient_Broadcast_Call { + _c.Call.Return(run) + 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 { + mock.TestingT + Cleanup(func()) +}) *MockStreamingCoordBroadcastServiceClient { + mock := &MockStreamingCoordBroadcastServiceClient{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchClient.go b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchClient.go new file mode 100644 index 0000000000000..fbebad3fc6f1b --- /dev/null +++ b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchClient.go @@ -0,0 +1,430 @@ +// 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/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 new file mode 100644 index 0000000000000..d5e1a24f8a0a8 --- /dev/null +++ b/pkg/mocks/proto/mock_streamingpb/mock_StreamingCoordBroadcastService_WatchServer.go @@ -0,0 +1,406 @@ +// 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/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/mocks/streaming/util/mock_message/mock_ImmutableMessage.go b/pkg/mocks/streaming/util/mock_message/mock_ImmutableMessage.go index 7d13ac551b922..83d04062e3c7f 100644 --- a/pkg/mocks/streaming/util/mock_message/mock_ImmutableMessage.go +++ b/pkg/mocks/streaming/util/mock_message/mock_ImmutableMessage.go @@ -65,6 +65,53 @@ func (_c *MockImmutableMessage_BarrierTimeTick_Call) RunAndReturn(run func() uin return _c } +// BroadcastHeader provides a mock function with given fields: +func (_m *MockImmutableMessage) BroadcastHeader() *message.BroadcastHeader { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for BroadcastHeader") + } + + var r0 *message.BroadcastHeader + if rf, ok := ret.Get(0).(func() *message.BroadcastHeader); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*message.BroadcastHeader) + } + } + + return r0 +} + +// MockImmutableMessage_BroadcastHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastHeader' +type MockImmutableMessage_BroadcastHeader_Call struct { + *mock.Call +} + +// BroadcastHeader is a helper method to define mock.On call +func (_e *MockImmutableMessage_Expecter) BroadcastHeader() *MockImmutableMessage_BroadcastHeader_Call { + return &MockImmutableMessage_BroadcastHeader_Call{Call: _e.mock.On("BroadcastHeader")} +} + +func (_c *MockImmutableMessage_BroadcastHeader_Call) Run(run func()) *MockImmutableMessage_BroadcastHeader_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockImmutableMessage_BroadcastHeader_Call) Return(_a0 *message.BroadcastHeader) *MockImmutableMessage_BroadcastHeader_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockImmutableMessage_BroadcastHeader_Call) RunAndReturn(run func() *message.BroadcastHeader) *MockImmutableMessage_BroadcastHeader_Call { + _c.Call.Return(run) + return _c +} + // EstimateSize provides a mock function with given fields: func (_m *MockImmutableMessage) EstimateSize() int { ret := _m.Called() diff --git a/pkg/mocks/streaming/util/mock_message/mock_ImmutableTxnMessage.go b/pkg/mocks/streaming/util/mock_message/mock_ImmutableTxnMessage.go index 21f8b21c3a16a..f3031dfc5d5ec 100644 --- a/pkg/mocks/streaming/util/mock_message/mock_ImmutableTxnMessage.go +++ b/pkg/mocks/streaming/util/mock_message/mock_ImmutableTxnMessage.go @@ -112,6 +112,53 @@ func (_c *MockImmutableTxnMessage_Begin_Call) RunAndReturn(run func() message.Im return _c } +// BroadcastHeader provides a mock function with given fields: +func (_m *MockImmutableTxnMessage) BroadcastHeader() *message.BroadcastHeader { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for BroadcastHeader") + } + + var r0 *message.BroadcastHeader + if rf, ok := ret.Get(0).(func() *message.BroadcastHeader); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*message.BroadcastHeader) + } + } + + return r0 +} + +// MockImmutableTxnMessage_BroadcastHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastHeader' +type MockImmutableTxnMessage_BroadcastHeader_Call struct { + *mock.Call +} + +// BroadcastHeader is a helper method to define mock.On call +func (_e *MockImmutableTxnMessage_Expecter) BroadcastHeader() *MockImmutableTxnMessage_BroadcastHeader_Call { + return &MockImmutableTxnMessage_BroadcastHeader_Call{Call: _e.mock.On("BroadcastHeader")} +} + +func (_c *MockImmutableTxnMessage_BroadcastHeader_Call) Run(run func()) *MockImmutableTxnMessage_BroadcastHeader_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockImmutableTxnMessage_BroadcastHeader_Call) Return(_a0 *message.BroadcastHeader) *MockImmutableTxnMessage_BroadcastHeader_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockImmutableTxnMessage_BroadcastHeader_Call) RunAndReturn(run func() *message.BroadcastHeader) *MockImmutableTxnMessage_BroadcastHeader_Call { + _c.Call.Return(run) + return _c +} + // Commit provides a mock function with given fields: func (_m *MockImmutableTxnMessage) Commit() message.ImmutableMessage { ret := _m.Called() diff --git a/pkg/mocks/streaming/util/mock_message/mock_MutableMessage.go b/pkg/mocks/streaming/util/mock_message/mock_MutableMessage.go index c3fbdc02d43b7..d0f127fe6e659 100644 --- a/pkg/mocks/streaming/util/mock_message/mock_MutableMessage.go +++ b/pkg/mocks/streaming/util/mock_message/mock_MutableMessage.go @@ -65,6 +65,53 @@ func (_c *MockMutableMessage_BarrierTimeTick_Call) RunAndReturn(run func() uint6 return _c } +// BroadcastHeader provides a mock function with given fields: +func (_m *MockMutableMessage) BroadcastHeader() *message.BroadcastHeader { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for BroadcastHeader") + } + + var r0 *message.BroadcastHeader + if rf, ok := ret.Get(0).(func() *message.BroadcastHeader); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*message.BroadcastHeader) + } + } + + return r0 +} + +// MockMutableMessage_BroadcastHeader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BroadcastHeader' +type MockMutableMessage_BroadcastHeader_Call struct { + *mock.Call +} + +// BroadcastHeader is a helper method to define mock.On call +func (_e *MockMutableMessage_Expecter) BroadcastHeader() *MockMutableMessage_BroadcastHeader_Call { + return &MockMutableMessage_BroadcastHeader_Call{Call: _e.mock.On("BroadcastHeader")} +} + +func (_c *MockMutableMessage_BroadcastHeader_Call) Run(run func()) *MockMutableMessage_BroadcastHeader_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockMutableMessage_BroadcastHeader_Call) Return(_a0 *message.BroadcastHeader) *MockMutableMessage_BroadcastHeader_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockMutableMessage_BroadcastHeader_Call) RunAndReturn(run func() *message.BroadcastHeader) *MockMutableMessage_BroadcastHeader_Call { + _c.Call.Return(run) + return _c +} + // EstimateSize provides a mock function with given fields: func (_m *MockMutableMessage) EstimateSize() int { ret := _m.Called() diff --git a/pkg/proto/messages.proto b/pkg/proto/messages.proto index 7ba14b70c42bd..61b525deb9387 100644 --- a/pkg/proto/messages.proto +++ b/pkg/proto/messages.proto @@ -249,7 +249,45 @@ message RMQMessageLayout { map properties = 2; // message properties } -// VChannels is a layout to represent the virtual channels for broadcast. -message VChannels { - repeated string vchannels = 1; -} \ No newline at end of file +// BroadcastHeader is the common header of broadcast message. +message BroadcastHeader { + uint64 broadcast_id = 1; + repeated string vchannels = 2; + repeated ResourceKey Resource_keys = 3; // the resource key of the broadcast message. + // Once the broadcast is sent, the resource of resource key will be hold. + // New broadcast message with the same resource key will be rejected. + // And the user can watch the resource key to known when the resource is released. +} + +// ResourceDomain is the domain of resource hold. +enum ResourceDomain { + ResourceDomainUnknown = 0; // should never be used. + ResourceDomainImportJobID = 1; // the domain of import job id. + ResourceDomainCollectionName = 2; // the domain of collection name. +} + +// ResourceKey is the key for resource hold. +// It's used to implement the resource acquirition mechanism for broadcast message. +// The key should be a unique identifier of the resource for different domain. +message ResourceKey { + ResourceDomain domain = 1; + string key = 2; +} + +// BroadcastEvent is the event of broadcast message. +message BroadcastEvent { + oneof event { + BroadcastResourceKeyAckAll resource_key_ack_all = 1; + BroadcastResourceKeyAckOne resource_key_ack_one = 2; + } +} + +// The resource key is acked by all vchannels. +message BroadcastResourceKeyAckAll { + messages.ResourceKey resource_key = 1; +} + +// The resource key is acked by any vchannel. +message BroadcastResourceKeyAckOne { + messages.ResourceKey resource_key = 1; +} diff --git a/pkg/proto/messagespb/messages.pb.go b/pkg/proto/messagespb/messages.pb.go index 809c0f1b2daab..0322d682e20e2 100644 --- a/pkg/proto/messagespb/messages.pb.go +++ b/pkg/proto/messagespb/messages.pb.go @@ -191,6 +191,56 @@ func (TxnState) EnumDescriptor() ([]byte, []int) { return file_messages_proto_rawDescGZIP(), []int{1} } +// ResourceDomain is the domain of resource hold. +type ResourceDomain int32 + +const ( + ResourceDomain_ResourceDomainUnknown ResourceDomain = 0 // should never be used. + ResourceDomain_ResourceDomainImportJobID ResourceDomain = 1 // the domain of import job id. + ResourceDomain_ResourceDomainCollectionName ResourceDomain = 2 // the domain of collection name. +) + +// Enum value maps for ResourceDomain. +var ( + ResourceDomain_name = map[int32]string{ + 0: "ResourceDomainUnknown", + 1: "ResourceDomainImportJobID", + 2: "ResourceDomainCollectionName", + } + ResourceDomain_value = map[string]int32{ + "ResourceDomainUnknown": 0, + "ResourceDomainImportJobID": 1, + "ResourceDomainCollectionName": 2, + } +) + +func (x ResourceDomain) Enum() *ResourceDomain { + p := new(ResourceDomain) + *p = x + return p +} + +func (x ResourceDomain) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ResourceDomain) Descriptor() protoreflect.EnumDescriptor { + return file_messages_proto_enumTypes[2].Descriptor() +} + +func (ResourceDomain) Type() protoreflect.EnumType { + return &file_messages_proto_enumTypes[2] +} + +func (x ResourceDomain) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ResourceDomain.Descriptor instead. +func (ResourceDomain) EnumDescriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{2} +} + // MessageID is the unique identifier of a message. type MessageID struct { state protoimpl.MessageState @@ -1689,17 +1739,19 @@ func (x *RMQMessageLayout) GetProperties() map[string]string { return nil } -// VChannels is a layout to represent the virtual channels for broadcast. -type VChannels struct { +// BroadcastHeader is the common header of broadcast message. +type BroadcastHeader struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Vchannels []string `protobuf:"bytes,1,rep,name=vchannels,proto3" json:"vchannels,omitempty"` + BroadcastId uint64 `protobuf:"varint,1,opt,name=broadcast_id,json=broadcastId,proto3" json:"broadcast_id,omitempty"` + Vchannels []string `protobuf:"bytes,2,rep,name=vchannels,proto3" json:"vchannels,omitempty"` + ResourceKeys []*ResourceKey `protobuf:"bytes,3,rep,name=Resource_keys,json=ResourceKeys,proto3" json:"Resource_keys,omitempty"` // the resource key of the broadcast message. } -func (x *VChannels) Reset() { - *x = VChannels{} +func (x *BroadcastHeader) Reset() { + *x = BroadcastHeader{} if protoimpl.UnsafeEnabled { mi := &file_messages_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -1707,13 +1759,13 @@ func (x *VChannels) Reset() { } } -func (x *VChannels) String() string { +func (x *BroadcastHeader) String() string { return protoimpl.X.MessageStringOf(x) } -func (*VChannels) ProtoMessage() {} +func (*BroadcastHeader) ProtoMessage() {} -func (x *VChannels) ProtoReflect() protoreflect.Message { +func (x *BroadcastHeader) ProtoReflect() protoreflect.Message { mi := &file_messages_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -1725,18 +1777,268 @@ func (x *VChannels) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use VChannels.ProtoReflect.Descriptor instead. -func (*VChannels) Descriptor() ([]byte, []int) { +// Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead. +func (*BroadcastHeader) Descriptor() ([]byte, []int) { return file_messages_proto_rawDescGZIP(), []int{30} } -func (x *VChannels) GetVchannels() []string { +func (x *BroadcastHeader) GetBroadcastId() uint64 { + if x != nil { + return x.BroadcastId + } + return 0 +} + +func (x *BroadcastHeader) GetVchannels() []string { if x != nil { return x.Vchannels } return nil } +func (x *BroadcastHeader) GetResourceKeys() []*ResourceKey { + if x != nil { + return x.ResourceKeys + } + return nil +} + +// ResourceKey is the key for resource hold. +// It's used to implement the resource acquirition mechanism for broadcast message. +// The key should be a unique identifier of the resource for different domain. +type ResourceKey struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Domain ResourceDomain `protobuf:"varint,1,opt,name=domain,proto3,enum=milvus.proto.messages.ResourceDomain" json:"domain,omitempty"` + Key string `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` +} + +func (x *ResourceKey) Reset() { + *x = ResourceKey{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResourceKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceKey) ProtoMessage() {} + +func (x *ResourceKey) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[31] + 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 ResourceKey.ProtoReflect.Descriptor instead. +func (*ResourceKey) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{31} +} + +func (x *ResourceKey) GetDomain() ResourceDomain { + if x != nil { + return x.Domain + } + return ResourceDomain_ResourceDomainUnknown +} + +func (x *ResourceKey) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +// BroadcastEvent is the event of broadcast message. +type BroadcastEvent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Event: + // + // *BroadcastEvent_ResourceKeyAckAll + // *BroadcastEvent_ResourceKeyAckOne + Event isBroadcastEvent_Event `protobuf_oneof:"event"` +} + +func (x *BroadcastEvent) Reset() { + *x = BroadcastEvent{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastEvent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastEvent) ProtoMessage() {} + +func (x *BroadcastEvent) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[32] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BroadcastEvent.ProtoReflect.Descriptor instead. +func (*BroadcastEvent) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{32} +} + +func (m *BroadcastEvent) GetEvent() isBroadcastEvent_Event { + if m != nil { + return m.Event + } + return nil +} + +func (x *BroadcastEvent) GetResourceKeyAckAll() *BroadcastResourceKeyAckAll { + if x, ok := x.GetEvent().(*BroadcastEvent_ResourceKeyAckAll); ok { + return x.ResourceKeyAckAll + } + return nil +} + +func (x *BroadcastEvent) GetResourceKeyAckOne() *BroadcastResourceKeyAckOne { + if x, ok := x.GetEvent().(*BroadcastEvent_ResourceKeyAckOne); ok { + return x.ResourceKeyAckOne + } + return nil +} + +type isBroadcastEvent_Event interface { + isBroadcastEvent_Event() +} + +type BroadcastEvent_ResourceKeyAckAll struct { + ResourceKeyAckAll *BroadcastResourceKeyAckAll `protobuf:"bytes,1,opt,name=resource_key_ack_all,json=resourceKeyAckAll,proto3,oneof"` +} + +type BroadcastEvent_ResourceKeyAckOne struct { + ResourceKeyAckOne *BroadcastResourceKeyAckOne `protobuf:"bytes,2,opt,name=resource_key_ack_one,json=resourceKeyAckOne,proto3,oneof"` +} + +func (*BroadcastEvent_ResourceKeyAckAll) isBroadcastEvent_Event() {} + +func (*BroadcastEvent_ResourceKeyAckOne) isBroadcastEvent_Event() {} + +// The resource key is acked by all vchannels. +type BroadcastResourceKeyAckAll struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ResourceKey *ResourceKey `protobuf:"bytes,1,opt,name=resource_key,json=resourceKey,proto3" json:"resource_key,omitempty"` +} + +func (x *BroadcastResourceKeyAckAll) Reset() { + *x = BroadcastResourceKeyAckAll{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastResourceKeyAckAll) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastResourceKeyAckAll) ProtoMessage() {} + +func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[33] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BroadcastResourceKeyAckAll.ProtoReflect.Descriptor instead. +func (*BroadcastResourceKeyAckAll) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{33} +} + +func (x *BroadcastResourceKeyAckAll) GetResourceKey() *ResourceKey { + if x != nil { + return x.ResourceKey + } + return nil +} + +// The resource key is acked by any vchannel. +type BroadcastResourceKeyAckOne struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ResourceKey *ResourceKey `protobuf:"bytes,1,opt,name=resource_key,json=resourceKey,proto3" json:"resource_key,omitempty"` +} + +func (x *BroadcastResourceKeyAckOne) Reset() { + *x = BroadcastResourceKeyAckOne{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastResourceKeyAckOne) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastResourceKeyAckOne) ProtoMessage() {} + +func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[34] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BroadcastResourceKeyAckOne.ProtoReflect.Descriptor instead. +func (*BroadcastResourceKeyAckOne) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{34} +} + +func (x *BroadcastResourceKeyAckOne) GetResourceKey() *ResourceKey { + if x != nil { + return x.ResourceKey + } + return nil +} + var File_messages_proto protoreflect.FileDescriptor var file_messages_proto_rawDesc = []byte{ @@ -1896,38 +2198,85 @@ var file_messages_proto_rawDesc = []byte{ 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x29, 0x0a, 0x09, 0x56, 0x43, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x2a, 0xfc, 0x01, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, - 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, - 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, - 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, - 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, - 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, - 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, - 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, - 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, - 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, - 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, - 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, - 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, - 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, - 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, - 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, - 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x42, 0x32, 0x5a, 0x30, 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, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, + 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, + 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, + 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 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, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, + 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, + 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, + 0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, + 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, + 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, + 0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, + 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45, + 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, + 0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x2a, 0xfc, 0x01, 0x0a, 0x0b, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, + 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, + 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, + 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, + 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, + 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, + 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0d, + 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, + 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, + 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, + 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, + 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, + 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, + 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, + 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, + 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, + 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, + 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, + 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, + 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x32, 0x5a, 0x30, + 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, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1942,60 +2291,71 @@ func file_messages_proto_rawDescGZIP() []byte { return file_messages_proto_rawDescData } -var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 34) +var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 38) var file_messages_proto_goTypes = []interface{}{ (MessageType)(0), // 0: milvus.proto.messages.MessageType (TxnState)(0), // 1: milvus.proto.messages.TxnState - (*MessageID)(nil), // 2: milvus.proto.messages.MessageID - (*Message)(nil), // 3: milvus.proto.messages.Message - (*ImmutableMessage)(nil), // 4: milvus.proto.messages.ImmutableMessage - (*FlushMessageBody)(nil), // 5: milvus.proto.messages.FlushMessageBody - (*ManualFlushMessageBody)(nil), // 6: milvus.proto.messages.ManualFlushMessageBody - (*CreateSegmentMessageBody)(nil), // 7: milvus.proto.messages.CreateSegmentMessageBody - (*CreateSegmentInfo)(nil), // 8: milvus.proto.messages.CreateSegmentInfo - (*BeginTxnMessageBody)(nil), // 9: milvus.proto.messages.BeginTxnMessageBody - (*CommitTxnMessageBody)(nil), // 10: milvus.proto.messages.CommitTxnMessageBody - (*RollbackTxnMessageBody)(nil), // 11: milvus.proto.messages.RollbackTxnMessageBody - (*TxnMessageBody)(nil), // 12: milvus.proto.messages.TxnMessageBody - (*TimeTickMessageHeader)(nil), // 13: milvus.proto.messages.TimeTickMessageHeader - (*InsertMessageHeader)(nil), // 14: milvus.proto.messages.InsertMessageHeader - (*PartitionSegmentAssignment)(nil), // 15: milvus.proto.messages.PartitionSegmentAssignment - (*SegmentAssignment)(nil), // 16: milvus.proto.messages.SegmentAssignment - (*DeleteMessageHeader)(nil), // 17: milvus.proto.messages.DeleteMessageHeader - (*FlushMessageHeader)(nil), // 18: milvus.proto.messages.FlushMessageHeader - (*CreateSegmentMessageHeader)(nil), // 19: milvus.proto.messages.CreateSegmentMessageHeader - (*ManualFlushMessageHeader)(nil), // 20: milvus.proto.messages.ManualFlushMessageHeader - (*CreateCollectionMessageHeader)(nil), // 21: milvus.proto.messages.CreateCollectionMessageHeader - (*DropCollectionMessageHeader)(nil), // 22: milvus.proto.messages.DropCollectionMessageHeader - (*CreatePartitionMessageHeader)(nil), // 23: milvus.proto.messages.CreatePartitionMessageHeader - (*DropPartitionMessageHeader)(nil), // 24: milvus.proto.messages.DropPartitionMessageHeader - (*BeginTxnMessageHeader)(nil), // 25: milvus.proto.messages.BeginTxnMessageHeader - (*CommitTxnMessageHeader)(nil), // 26: milvus.proto.messages.CommitTxnMessageHeader - (*RollbackTxnMessageHeader)(nil), // 27: milvus.proto.messages.RollbackTxnMessageHeader - (*TxnMessageHeader)(nil), // 28: milvus.proto.messages.TxnMessageHeader - (*ManualFlushExtraResponse)(nil), // 29: milvus.proto.messages.ManualFlushExtraResponse - (*TxnContext)(nil), // 30: milvus.proto.messages.TxnContext - (*RMQMessageLayout)(nil), // 31: milvus.proto.messages.RMQMessageLayout - (*VChannels)(nil), // 32: milvus.proto.messages.VChannels - nil, // 33: milvus.proto.messages.Message.PropertiesEntry - nil, // 34: milvus.proto.messages.ImmutableMessage.PropertiesEntry - nil, // 35: milvus.proto.messages.RMQMessageLayout.PropertiesEntry + (ResourceDomain)(0), // 2: milvus.proto.messages.ResourceDomain + (*MessageID)(nil), // 3: milvus.proto.messages.MessageID + (*Message)(nil), // 4: milvus.proto.messages.Message + (*ImmutableMessage)(nil), // 5: milvus.proto.messages.ImmutableMessage + (*FlushMessageBody)(nil), // 6: milvus.proto.messages.FlushMessageBody + (*ManualFlushMessageBody)(nil), // 7: milvus.proto.messages.ManualFlushMessageBody + (*CreateSegmentMessageBody)(nil), // 8: milvus.proto.messages.CreateSegmentMessageBody + (*CreateSegmentInfo)(nil), // 9: milvus.proto.messages.CreateSegmentInfo + (*BeginTxnMessageBody)(nil), // 10: milvus.proto.messages.BeginTxnMessageBody + (*CommitTxnMessageBody)(nil), // 11: milvus.proto.messages.CommitTxnMessageBody + (*RollbackTxnMessageBody)(nil), // 12: milvus.proto.messages.RollbackTxnMessageBody + (*TxnMessageBody)(nil), // 13: milvus.proto.messages.TxnMessageBody + (*TimeTickMessageHeader)(nil), // 14: milvus.proto.messages.TimeTickMessageHeader + (*InsertMessageHeader)(nil), // 15: milvus.proto.messages.InsertMessageHeader + (*PartitionSegmentAssignment)(nil), // 16: milvus.proto.messages.PartitionSegmentAssignment + (*SegmentAssignment)(nil), // 17: milvus.proto.messages.SegmentAssignment + (*DeleteMessageHeader)(nil), // 18: milvus.proto.messages.DeleteMessageHeader + (*FlushMessageHeader)(nil), // 19: milvus.proto.messages.FlushMessageHeader + (*CreateSegmentMessageHeader)(nil), // 20: milvus.proto.messages.CreateSegmentMessageHeader + (*ManualFlushMessageHeader)(nil), // 21: milvus.proto.messages.ManualFlushMessageHeader + (*CreateCollectionMessageHeader)(nil), // 22: milvus.proto.messages.CreateCollectionMessageHeader + (*DropCollectionMessageHeader)(nil), // 23: milvus.proto.messages.DropCollectionMessageHeader + (*CreatePartitionMessageHeader)(nil), // 24: milvus.proto.messages.CreatePartitionMessageHeader + (*DropPartitionMessageHeader)(nil), // 25: milvus.proto.messages.DropPartitionMessageHeader + (*BeginTxnMessageHeader)(nil), // 26: milvus.proto.messages.BeginTxnMessageHeader + (*CommitTxnMessageHeader)(nil), // 27: milvus.proto.messages.CommitTxnMessageHeader + (*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader + (*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader + (*ManualFlushExtraResponse)(nil), // 30: milvus.proto.messages.ManualFlushExtraResponse + (*TxnContext)(nil), // 31: milvus.proto.messages.TxnContext + (*RMQMessageLayout)(nil), // 32: milvus.proto.messages.RMQMessageLayout + (*BroadcastHeader)(nil), // 33: milvus.proto.messages.BroadcastHeader + (*ResourceKey)(nil), // 34: milvus.proto.messages.ResourceKey + (*BroadcastEvent)(nil), // 35: milvus.proto.messages.BroadcastEvent + (*BroadcastResourceKeyAckAll)(nil), // 36: milvus.proto.messages.BroadcastResourceKeyAckAll + (*BroadcastResourceKeyAckOne)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckOne + nil, // 38: milvus.proto.messages.Message.PropertiesEntry + nil, // 39: milvus.proto.messages.ImmutableMessage.PropertiesEntry + nil, // 40: milvus.proto.messages.RMQMessageLayout.PropertiesEntry } var file_messages_proto_depIdxs = []int32{ - 33, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry - 2, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID - 34, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry - 8, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo - 3, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message - 15, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment - 16, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment - 35, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry - 8, // [8:8] is the sub-list for method output_type - 8, // [8:8] is the sub-list for method input_type - 8, // [8:8] is the sub-list for extension type_name - 8, // [8:8] is the sub-list for extension extendee - 0, // [0:8] is the sub-list for field type_name + 38, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry + 3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID + 39, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry + 9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo + 4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message + 16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment + 17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment + 40, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry + 34, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey + 2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain + 36, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll + 37, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne + 34, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey + 34, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey + 14, // [14:14] is the sub-list for method output_type + 14, // [14:14] is the sub-list for method input_type + 14, // [14:14] is the sub-list for extension type_name + 14, // [14:14] is the sub-list for extension extendee + 0, // [0:14] is the sub-list for field type_name } func init() { file_messages_proto_init() } @@ -2365,7 +2725,19 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VChannels); i { + switch v := v.(*BroadcastHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResourceKey); i { case 0: return &v.state case 1: @@ -2376,14 +2748,54 @@ func file_messages_proto_init() { return nil } } + file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastEvent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastResourceKeyAckAll); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastResourceKeyAckOne); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_messages_proto_msgTypes[32].OneofWrappers = []interface{}{ + (*BroadcastEvent_ResourceKeyAckAll)(nil), + (*BroadcastEvent_ResourceKeyAckOne)(nil), } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_messages_proto_rawDesc, - NumEnums: 2, - NumMessages: 34, + NumEnums: 3, + NumMessages: 38, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/streaming.proto b/pkg/proto/streaming.proto index 4539bf34395ce..3e83ee910f8c9 100644 --- a/pkg/proto/streaming.proto +++ b/pkg/proto/streaming.proto @@ -62,16 +62,17 @@ message VersionPair { // BroadcastTaskState is the state of the broadcast task. enum BroadcastTaskState { - BROADCAST_TASK_STATE_UNKNOWN = 0; // should never used. - BROADCAST_TASK_STATE_PENDING = 1; // task is pending. - BROADCAST_TASK_STATE_DONE = 2; // task is done, the message is broadcasted, and the persisted task can be cleared. + BROADCAST_TASK_STATE_UNKNOWN = 0; // should never used. + BROADCAST_TASK_STATE_PENDING = 1; // task is pending. + BROADCAST_TASK_STATE_DONE = 2; // task has been broadcasted and acknowledged, the resource lock is released, and the persisted task can be cleared. + BROADCAST_TASK_STATE_WAIT_ACK = 3; // task has been broadcasted, waiting for ack, the resource lock is still acquired by some vchannels. } -// BroadcastTask is the task to broadcast the message. +// BroadcastTask is the task to broadcast the messake. message BroadcastTask { - int64 task_id = 1; // task id. - messages.Message message = 2; // message to be broadcast. - BroadcastTaskState state = 3; // state of the task. + messages.Message message = 1; // message to be broadcast. + BroadcastTaskState state = 2; // state of the task. + bytes acked_vchannel_bitmap = 3; // given vchannels that have been acked, the size of bitmap is same with message.BroadcastHeader().VChannels. } // @@ -92,6 +93,12 @@ service StreamingCoordBroadcastService { // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. // It performs an atomic broadcast to all wal, achieve eventual consistency. rpc Broadcast(BroadcastRequest) returns (BroadcastResponse) {} + + // 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. @@ -102,6 +109,43 @@ message BroadcastRequest { // BroadcastResponse is the response of the Broadcast RPC. message BroadcastResponse { map results = 1; + uint64 broadcast_id = 2; +} + +message BroadcastAckRequest { + uint64 broadcast_id = 1; // broadcast id. + string vchannel = 2; // the vchannel that acked the message. +} + +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 { } // @@ -223,6 +267,7 @@ enum StreamingCode { STREAMING_CODE_TRANSACTION_EXPIRED = 9; // transaction expired STREAMING_CODE_INVALID_TRANSACTION_STATE = 10; // invalid transaction state STREAMING_CODE_UNRECOVERABLE = 11; // unrecoverable error + STREAMING_CODE_RESOURCE_ACQUIRED = 12; // resource is acquired by other operation STREAMING_CODE_UNKNOWN = 999; // unknown error } diff --git a/pkg/proto/streamingpb/streaming.pb.go b/pkg/proto/streamingpb/streaming.pb.go index 850806e5d41bb..f233cdb8a7973 100644 --- a/pkg/proto/streamingpb/streaming.pb.go +++ b/pkg/proto/streamingpb/streaming.pb.go @@ -84,9 +84,10 @@ func (PChannelMetaState) EnumDescriptor() ([]byte, []int) { type BroadcastTaskState int32 const ( - BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN BroadcastTaskState = 0 // should never used. - BroadcastTaskState_BROADCAST_TASK_STATE_PENDING BroadcastTaskState = 1 // task is pending. - BroadcastTaskState_BROADCAST_TASK_STATE_DONE BroadcastTaskState = 2 // task is done, the message is broadcasted, and the persisted task can be cleared. + BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN BroadcastTaskState = 0 // should never used. + BroadcastTaskState_BROADCAST_TASK_STATE_PENDING BroadcastTaskState = 1 // task is pending. + BroadcastTaskState_BROADCAST_TASK_STATE_DONE BroadcastTaskState = 2 // task has been broadcasted and acknowledged, the resource lock is released, and the persisted task can be cleared. + BroadcastTaskState_BROADCAST_TASK_STATE_WAIT_ACK BroadcastTaskState = 3 // task has been broadcasted, waiting for ack, the resource lock is still acquired by some vchannels. ) // Enum value maps for BroadcastTaskState. @@ -95,11 +96,13 @@ var ( 0: "BROADCAST_TASK_STATE_UNKNOWN", 1: "BROADCAST_TASK_STATE_PENDING", 2: "BROADCAST_TASK_STATE_DONE", + 3: "BROADCAST_TASK_STATE_WAIT_ACK", } BroadcastTaskState_value = map[string]int32{ - "BROADCAST_TASK_STATE_UNKNOWN": 0, - "BROADCAST_TASK_STATE_PENDING": 1, - "BROADCAST_TASK_STATE_DONE": 2, + "BROADCAST_TASK_STATE_UNKNOWN": 0, + "BROADCAST_TASK_STATE_PENDING": 1, + "BROADCAST_TASK_STATE_DONE": 2, + "BROADCAST_TASK_STATE_WAIT_ACK": 3, } ) @@ -146,6 +149,7 @@ const ( StreamingCode_STREAMING_CODE_TRANSACTION_EXPIRED StreamingCode = 9 // transaction expired StreamingCode_STREAMING_CODE_INVALID_TRANSACTION_STATE StreamingCode = 10 // invalid transaction state StreamingCode_STREAMING_CODE_UNRECOVERABLE StreamingCode = 11 // unrecoverable error + StreamingCode_STREAMING_CODE_RESOURCE_ACQUIRED StreamingCode = 12 // resource is acquired by other operation StreamingCode_STREAMING_CODE_UNKNOWN StreamingCode = 999 // unknown error ) @@ -164,6 +168,7 @@ var ( 9: "STREAMING_CODE_TRANSACTION_EXPIRED", 10: "STREAMING_CODE_INVALID_TRANSACTION_STATE", 11: "STREAMING_CODE_UNRECOVERABLE", + 12: "STREAMING_CODE_RESOURCE_ACQUIRED", 999: "STREAMING_CODE_UNKNOWN", } StreamingCode_value = map[string]int32{ @@ -179,6 +184,7 @@ var ( "STREAMING_CODE_TRANSACTION_EXPIRED": 9, "STREAMING_CODE_INVALID_TRANSACTION_STATE": 10, "STREAMING_CODE_UNRECOVERABLE": 11, + "STREAMING_CODE_RESOURCE_ACQUIRED": 12, "STREAMING_CODE_UNKNOWN": 999, } ) @@ -512,15 +518,15 @@ func (x *VersionPair) GetLocal() int64 { return 0 } -// BroadcastTask is the task to broadcast the message. +// BroadcastTask is the task to broadcast the messake. type BroadcastTask struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - TaskId int64 `protobuf:"varint,1,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` // task id. - Message *messagespb.Message `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` // message to be broadcast. - State BroadcastTaskState `protobuf:"varint,3,opt,name=state,proto3,enum=milvus.proto.streaming.BroadcastTaskState" json:"state,omitempty"` // state of the task. + Message *messagespb.Message `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` // message to be broadcast. + State BroadcastTaskState `protobuf:"varint,2,opt,name=state,proto3,enum=milvus.proto.streaming.BroadcastTaskState" json:"state,omitempty"` // state of the task. + AckedVchannelBitmap []byte `protobuf:"bytes,3,opt,name=acked_vchannel_bitmap,json=ackedVchannelBitmap,proto3" json:"acked_vchannel_bitmap,omitempty"` // given vchannels that have been acked, the size of bitmap is same with message.BroadcastHeader().VChannels. } func (x *BroadcastTask) Reset() { @@ -555,13 +561,6 @@ func (*BroadcastTask) Descriptor() ([]byte, []int) { return file_streaming_proto_rawDescGZIP(), []int{4} } -func (x *BroadcastTask) GetTaskId() int64 { - if x != nil { - return x.TaskId - } - return 0 -} - func (x *BroadcastTask) GetMessage() *messagespb.Message { if x != nil { return x.Message @@ -576,6 +575,13 @@ func (x *BroadcastTask) GetState() BroadcastTaskState { return BroadcastTaskState_BROADCAST_TASK_STATE_UNKNOWN } +func (x *BroadcastTask) GetAckedVchannelBitmap() []byte { + if x != nil { + return x.AckedVchannelBitmap + } + return nil +} + // BroadcastRequest is the request of the Broadcast RPC. type BroadcastRequest struct { state protoimpl.MessageState @@ -630,7 +636,8 @@ type BroadcastResponse struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Results map[string]*ProduceMessageResponseResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Results map[string]*ProduceMessageResponseResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + BroadcastId uint64 `protobuf:"varint,2,opt,name=broadcast_id,json=broadcastId,proto3" json:"broadcast_id,omitempty"` } func (x *BroadcastResponse) Reset() { @@ -672,6 +679,438 @@ func (x *BroadcastResponse) GetResults() map[string]*ProduceMessageResponseResul return nil } +func (x *BroadcastResponse) GetBroadcastId() uint64 { + if x != nil { + return x.BroadcastId + } + return 0 +} + +type BroadcastAckRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BroadcastId uint64 `protobuf:"varint,1,opt,name=broadcast_id,json=broadcastId,proto3" json:"broadcast_id,omitempty"` // broadcast id. + Vchannel string `protobuf:"bytes,2,opt,name=vchannel,proto3" json:"vchannel,omitempty"` // the vchannel that acked the message. +} + +func (x *BroadcastAckRequest) Reset() { + *x = BroadcastAckRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastAckRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastAckRequest) ProtoMessage() {} + +func (x *BroadcastAckRequest) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[7] + 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 BroadcastAckRequest.ProtoReflect.Descriptor instead. +func (*BroadcastAckRequest) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{7} +} + +func (x *BroadcastAckRequest) GetBroadcastId() uint64 { + if x != nil { + return x.BroadcastId + } + return 0 +} + +func (x *BroadcastAckRequest) GetVchannel() string { + if x != nil { + return x.Vchannel + } + return "" +} + +type BroadcastAckResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *BroadcastAckResponse) Reset() { + *x = BroadcastAckResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastAckResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastAckResponse) ProtoMessage() {} + +func (x *BroadcastAckResponse) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[8] + 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 BroadcastAckResponse.ProtoReflect.Descriptor instead. +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 @@ -688,7 +1127,7 @@ type AssignmentDiscoverRequest struct { func (x *AssignmentDiscoverRequest) Reset() { *x = AssignmentDiscoverRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[7] + mi := &file_streaming_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -701,7 +1140,7 @@ func (x *AssignmentDiscoverRequest) String() string { func (*AssignmentDiscoverRequest) ProtoMessage() {} func (x *AssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[7] + mi := &file_streaming_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -714,7 +1153,7 @@ func (x *AssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignmentDiscoverRequest.ProtoReflect.Descriptor instead. func (*AssignmentDiscoverRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{7} + return file_streaming_proto_rawDescGZIP(), []int{15} } func (m *AssignmentDiscoverRequest) GetCommand() isAssignmentDiscoverRequest_Command { @@ -768,7 +1207,7 @@ type ReportAssignmentErrorRequest struct { func (x *ReportAssignmentErrorRequest) Reset() { *x = ReportAssignmentErrorRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[8] + mi := &file_streaming_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -781,7 +1220,7 @@ func (x *ReportAssignmentErrorRequest) String() string { func (*ReportAssignmentErrorRequest) ProtoMessage() {} func (x *ReportAssignmentErrorRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[8] + mi := &file_streaming_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -794,7 +1233,7 @@ func (x *ReportAssignmentErrorRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReportAssignmentErrorRequest.ProtoReflect.Descriptor instead. func (*ReportAssignmentErrorRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{8} + return file_streaming_proto_rawDescGZIP(), []int{16} } func (x *ReportAssignmentErrorRequest) GetPchannel() *PChannelInfo { @@ -821,7 +1260,7 @@ type CloseAssignmentDiscoverRequest struct { func (x *CloseAssignmentDiscoverRequest) Reset() { *x = CloseAssignmentDiscoverRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[9] + mi := &file_streaming_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -834,7 +1273,7 @@ func (x *CloseAssignmentDiscoverRequest) String() string { func (*CloseAssignmentDiscoverRequest) ProtoMessage() {} func (x *CloseAssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[9] + mi := &file_streaming_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -847,7 +1286,7 @@ func (x *CloseAssignmentDiscoverRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseAssignmentDiscoverRequest.ProtoReflect.Descriptor instead. func (*CloseAssignmentDiscoverRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{9} + return file_streaming_proto_rawDescGZIP(), []int{17} } // AssignmentDiscoverResponse is the response of Discovery @@ -866,7 +1305,7 @@ type AssignmentDiscoverResponse struct { func (x *AssignmentDiscoverResponse) Reset() { *x = AssignmentDiscoverResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[10] + mi := &file_streaming_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -879,7 +1318,7 @@ func (x *AssignmentDiscoverResponse) String() string { func (*AssignmentDiscoverResponse) ProtoMessage() {} func (x *AssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[10] + mi := &file_streaming_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -892,7 +1331,7 @@ func (x *AssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignmentDiscoverResponse.ProtoReflect.Descriptor instead. func (*AssignmentDiscoverResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{10} + return file_streaming_proto_rawDescGZIP(), []int{18} } func (m *AssignmentDiscoverResponse) GetResponse() isAssignmentDiscoverResponse_Response { @@ -947,7 +1386,7 @@ type FullStreamingNodeAssignmentWithVersion struct { func (x *FullStreamingNodeAssignmentWithVersion) Reset() { *x = FullStreamingNodeAssignmentWithVersion{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[11] + mi := &file_streaming_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -960,7 +1399,7 @@ func (x *FullStreamingNodeAssignmentWithVersion) String() string { func (*FullStreamingNodeAssignmentWithVersion) ProtoMessage() {} func (x *FullStreamingNodeAssignmentWithVersion) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[11] + mi := &file_streaming_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -973,7 +1412,7 @@ func (x *FullStreamingNodeAssignmentWithVersion) ProtoReflect() protoreflect.Mes // Deprecated: Use FullStreamingNodeAssignmentWithVersion.ProtoReflect.Descriptor instead. func (*FullStreamingNodeAssignmentWithVersion) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{11} + return file_streaming_proto_rawDescGZIP(), []int{19} } func (x *FullStreamingNodeAssignmentWithVersion) GetVersion() *VersionPair { @@ -999,7 +1438,7 @@ type CloseAssignmentDiscoverResponse struct { func (x *CloseAssignmentDiscoverResponse) Reset() { *x = CloseAssignmentDiscoverResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[12] + mi := &file_streaming_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1012,7 +1451,7 @@ func (x *CloseAssignmentDiscoverResponse) String() string { func (*CloseAssignmentDiscoverResponse) ProtoMessage() {} func (x *CloseAssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[12] + mi := &file_streaming_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1025,7 +1464,7 @@ func (x *CloseAssignmentDiscoverResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseAssignmentDiscoverResponse.ProtoReflect.Descriptor instead. func (*CloseAssignmentDiscoverResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{12} + return file_streaming_proto_rawDescGZIP(), []int{20} } // StreamingNodeInfo is the information of a streaming node. @@ -1041,7 +1480,7 @@ type StreamingNodeInfo struct { func (x *StreamingNodeInfo) Reset() { *x = StreamingNodeInfo{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[13] + mi := &file_streaming_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1054,7 +1493,7 @@ func (x *StreamingNodeInfo) String() string { func (*StreamingNodeInfo) ProtoMessage() {} func (x *StreamingNodeInfo) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[13] + mi := &file_streaming_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1067,7 +1506,7 @@ func (x *StreamingNodeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingNodeInfo.ProtoReflect.Descriptor instead. func (*StreamingNodeInfo) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{13} + return file_streaming_proto_rawDescGZIP(), []int{21} } func (x *StreamingNodeInfo) GetServerId() int64 { @@ -1097,7 +1536,7 @@ type StreamingNodeAssignment struct { func (x *StreamingNodeAssignment) Reset() { *x = StreamingNodeAssignment{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[14] + mi := &file_streaming_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1110,7 +1549,7 @@ func (x *StreamingNodeAssignment) String() string { func (*StreamingNodeAssignment) ProtoMessage() {} func (x *StreamingNodeAssignment) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[14] + mi := &file_streaming_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1123,7 +1562,7 @@ func (x *StreamingNodeAssignment) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingNodeAssignment.ProtoReflect.Descriptor instead. func (*StreamingNodeAssignment) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{14} + return file_streaming_proto_rawDescGZIP(), []int{22} } func (x *StreamingNodeAssignment) GetNode() *StreamingNodeInfo { @@ -1158,7 +1597,7 @@ type DeliverPolicy struct { func (x *DeliverPolicy) Reset() { *x = DeliverPolicy{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[15] + mi := &file_streaming_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1171,7 +1610,7 @@ func (x *DeliverPolicy) String() string { func (*DeliverPolicy) ProtoMessage() {} func (x *DeliverPolicy) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[15] + mi := &file_streaming_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1184,7 +1623,7 @@ func (x *DeliverPolicy) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverPolicy.ProtoReflect.Descriptor instead. func (*DeliverPolicy) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{15} + return file_streaming_proto_rawDescGZIP(), []int{23} } func (m *DeliverPolicy) GetPolicy() isDeliverPolicy_Policy { @@ -1267,7 +1706,7 @@ type DeliverFilter struct { func (x *DeliverFilter) Reset() { *x = DeliverFilter{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[16] + mi := &file_streaming_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1280,7 +1719,7 @@ func (x *DeliverFilter) String() string { func (*DeliverFilter) ProtoMessage() {} func (x *DeliverFilter) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[16] + mi := &file_streaming_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1293,7 +1732,7 @@ func (x *DeliverFilter) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilter.ProtoReflect.Descriptor instead. func (*DeliverFilter) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{16} + return file_streaming_proto_rawDescGZIP(), []int{24} } func (m *DeliverFilter) GetFilter() isDeliverFilter_Filter { @@ -1359,7 +1798,7 @@ type DeliverFilterTimeTickGT struct { func (x *DeliverFilterTimeTickGT) Reset() { *x = DeliverFilterTimeTickGT{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[17] + mi := &file_streaming_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1372,7 +1811,7 @@ func (x *DeliverFilterTimeTickGT) String() string { func (*DeliverFilterTimeTickGT) ProtoMessage() {} func (x *DeliverFilterTimeTickGT) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[17] + mi := &file_streaming_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1385,7 +1824,7 @@ func (x *DeliverFilterTimeTickGT) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilterTimeTickGT.ProtoReflect.Descriptor instead. func (*DeliverFilterTimeTickGT) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{17} + return file_streaming_proto_rawDescGZIP(), []int{25} } func (x *DeliverFilterTimeTickGT) GetTimeTick() uint64 { @@ -1408,7 +1847,7 @@ type DeliverFilterTimeTickGTE struct { func (x *DeliverFilterTimeTickGTE) Reset() { *x = DeliverFilterTimeTickGTE{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[18] + mi := &file_streaming_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1421,7 +1860,7 @@ func (x *DeliverFilterTimeTickGTE) String() string { func (*DeliverFilterTimeTickGTE) ProtoMessage() {} func (x *DeliverFilterTimeTickGTE) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[18] + mi := &file_streaming_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1434,7 +1873,7 @@ func (x *DeliverFilterTimeTickGTE) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilterTimeTickGTE.ProtoReflect.Descriptor instead. func (*DeliverFilterTimeTickGTE) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{18} + return file_streaming_proto_rawDescGZIP(), []int{26} } func (x *DeliverFilterTimeTickGTE) GetTimeTick() uint64 { @@ -1456,7 +1895,7 @@ type DeliverFilterMessageType struct { func (x *DeliverFilterMessageType) Reset() { *x = DeliverFilterMessageType{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[19] + mi := &file_streaming_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1469,7 +1908,7 @@ func (x *DeliverFilterMessageType) String() string { func (*DeliverFilterMessageType) ProtoMessage() {} func (x *DeliverFilterMessageType) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[19] + mi := &file_streaming_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1482,7 +1921,7 @@ func (x *DeliverFilterMessageType) ProtoReflect() protoreflect.Message { // Deprecated: Use DeliverFilterMessageType.ProtoReflect.Descriptor instead. func (*DeliverFilterMessageType) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{19} + return file_streaming_proto_rawDescGZIP(), []int{27} } func (x *DeliverFilterMessageType) GetMessageTypes() []messagespb.MessageType { @@ -1505,7 +1944,7 @@ type StreamingError struct { func (x *StreamingError) Reset() { *x = StreamingError{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[20] + mi := &file_streaming_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1518,7 +1957,7 @@ func (x *StreamingError) String() string { func (*StreamingError) ProtoMessage() {} func (x *StreamingError) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[20] + mi := &file_streaming_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1531,7 +1970,7 @@ func (x *StreamingError) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingError.ProtoReflect.Descriptor instead. func (*StreamingError) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{20} + return file_streaming_proto_rawDescGZIP(), []int{28} } func (x *StreamingError) GetCode() StreamingCode { @@ -1566,7 +2005,7 @@ type ProduceRequest struct { func (x *ProduceRequest) Reset() { *x = ProduceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[21] + mi := &file_streaming_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1579,7 +2018,7 @@ func (x *ProduceRequest) String() string { func (*ProduceRequest) ProtoMessage() {} func (x *ProduceRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[21] + mi := &file_streaming_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1592,7 +2031,7 @@ func (x *ProduceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceRequest.ProtoReflect.Descriptor instead. func (*ProduceRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{21} + return file_streaming_proto_rawDescGZIP(), []int{29} } func (m *ProduceRequest) GetRequest() isProduceRequest_Request { @@ -1645,7 +2084,7 @@ type CreateProducerRequest struct { func (x *CreateProducerRequest) Reset() { *x = CreateProducerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[22] + mi := &file_streaming_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1658,7 +2097,7 @@ func (x *CreateProducerRequest) String() string { func (*CreateProducerRequest) ProtoMessage() {} func (x *CreateProducerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[22] + mi := &file_streaming_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1671,7 +2110,7 @@ func (x *CreateProducerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateProducerRequest.ProtoReflect.Descriptor instead. func (*CreateProducerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{22} + return file_streaming_proto_rawDescGZIP(), []int{30} } func (x *CreateProducerRequest) GetPchannel() *PChannelInfo { @@ -1694,7 +2133,7 @@ type ProduceMessageRequest struct { func (x *ProduceMessageRequest) Reset() { *x = ProduceMessageRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[23] + mi := &file_streaming_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1707,7 +2146,7 @@ func (x *ProduceMessageRequest) String() string { func (*ProduceMessageRequest) ProtoMessage() {} func (x *ProduceMessageRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[23] + mi := &file_streaming_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1720,7 +2159,7 @@ func (x *ProduceMessageRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceMessageRequest.ProtoReflect.Descriptor instead. func (*ProduceMessageRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{23} + return file_streaming_proto_rawDescGZIP(), []int{31} } func (x *ProduceMessageRequest) GetRequestId() int64 { @@ -1748,7 +2187,7 @@ type CloseProducerRequest struct { func (x *CloseProducerRequest) Reset() { *x = CloseProducerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[24] + mi := &file_streaming_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1761,7 +2200,7 @@ func (x *CloseProducerRequest) String() string { func (*CloseProducerRequest) ProtoMessage() {} func (x *CloseProducerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[24] + mi := &file_streaming_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1774,7 +2213,7 @@ func (x *CloseProducerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseProducerRequest.ProtoReflect.Descriptor instead. func (*CloseProducerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{24} + return file_streaming_proto_rawDescGZIP(), []int{32} } // ProduceResponse is the response of the Produce RPC. @@ -1794,7 +2233,7 @@ type ProduceResponse struct { func (x *ProduceResponse) Reset() { *x = ProduceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[25] + mi := &file_streaming_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1807,7 +2246,7 @@ func (x *ProduceResponse) String() string { func (*ProduceResponse) ProtoMessage() {} func (x *ProduceResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[25] + mi := &file_streaming_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1820,7 +2259,7 @@ func (x *ProduceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceResponse.ProtoReflect.Descriptor instead. func (*ProduceResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{25} + return file_streaming_proto_rawDescGZIP(), []int{33} } func (m *ProduceResponse) GetResponse() isProduceResponse_Response { @@ -1886,7 +2325,7 @@ type CreateProducerResponse struct { func (x *CreateProducerResponse) Reset() { *x = CreateProducerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[26] + mi := &file_streaming_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1899,7 +2338,7 @@ func (x *CreateProducerResponse) String() string { func (*CreateProducerResponse) ProtoMessage() {} func (x *CreateProducerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[26] + mi := &file_streaming_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1912,7 +2351,7 @@ func (x *CreateProducerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateProducerResponse.ProtoReflect.Descriptor instead. func (*CreateProducerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{26} + return file_streaming_proto_rawDescGZIP(), []int{34} } func (x *CreateProducerResponse) GetWalName() string { @@ -1946,7 +2385,7 @@ type ProduceMessageResponse struct { func (x *ProduceMessageResponse) Reset() { *x = ProduceMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[27] + mi := &file_streaming_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1959,7 +2398,7 @@ func (x *ProduceMessageResponse) String() string { func (*ProduceMessageResponse) ProtoMessage() {} func (x *ProduceMessageResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[27] + mi := &file_streaming_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1972,7 +2411,7 @@ func (x *ProduceMessageResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceMessageResponse.ProtoReflect.Descriptor instead. func (*ProduceMessageResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{27} + return file_streaming_proto_rawDescGZIP(), []int{35} } func (x *ProduceMessageResponse) GetRequestId() int64 { @@ -2035,7 +2474,7 @@ type ProduceMessageResponseResult struct { func (x *ProduceMessageResponseResult) Reset() { *x = ProduceMessageResponseResult{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[28] + mi := &file_streaming_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2048,7 +2487,7 @@ func (x *ProduceMessageResponseResult) String() string { func (*ProduceMessageResponseResult) ProtoMessage() {} func (x *ProduceMessageResponseResult) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[28] + mi := &file_streaming_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2061,7 +2500,7 @@ func (x *ProduceMessageResponseResult) ProtoReflect() protoreflect.Message { // Deprecated: Use ProduceMessageResponseResult.ProtoReflect.Descriptor instead. func (*ProduceMessageResponseResult) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{28} + return file_streaming_proto_rawDescGZIP(), []int{36} } func (x *ProduceMessageResponseResult) GetId() *messagespb.MessageID { @@ -2102,7 +2541,7 @@ type CloseProducerResponse struct { func (x *CloseProducerResponse) Reset() { *x = CloseProducerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[29] + mi := &file_streaming_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2115,7 +2554,7 @@ func (x *CloseProducerResponse) String() string { func (*CloseProducerResponse) ProtoMessage() {} func (x *CloseProducerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[29] + mi := &file_streaming_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2128,7 +2567,7 @@ func (x *CloseProducerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseProducerResponse.ProtoReflect.Descriptor instead. func (*CloseProducerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{29} + return file_streaming_proto_rawDescGZIP(), []int{37} } // ConsumeRequest is the request of the Consume RPC. @@ -2150,7 +2589,7 @@ type ConsumeRequest struct { func (x *ConsumeRequest) Reset() { *x = ConsumeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[30] + mi := &file_streaming_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2163,7 +2602,7 @@ func (x *ConsumeRequest) String() string { func (*ConsumeRequest) ProtoMessage() {} func (x *ConsumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[30] + mi := &file_streaming_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2176,7 +2615,7 @@ func (x *ConsumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ConsumeRequest.ProtoReflect.Descriptor instead. func (*ConsumeRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{30} + return file_streaming_proto_rawDescGZIP(), []int{38} } func (m *ConsumeRequest) GetRequest() isConsumeRequest_Request { @@ -2253,7 +2692,7 @@ type CloseConsumerRequest struct { func (x *CloseConsumerRequest) Reset() { *x = CloseConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[31] + mi := &file_streaming_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2266,7 +2705,7 @@ func (x *CloseConsumerRequest) String() string { func (*CloseConsumerRequest) ProtoMessage() {} func (x *CloseConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[31] + mi := &file_streaming_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2279,7 +2718,7 @@ func (x *CloseConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseConsumerRequest.ProtoReflect.Descriptor instead. func (*CloseConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{31} + return file_streaming_proto_rawDescGZIP(), []int{39} } // CreateConsumerRequest is the request of the CreateConsumer RPC. @@ -2295,7 +2734,7 @@ type CreateConsumerRequest struct { func (x *CreateConsumerRequest) Reset() { *x = CreateConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[32] + mi := &file_streaming_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2308,7 +2747,7 @@ func (x *CreateConsumerRequest) String() string { func (*CreateConsumerRequest) ProtoMessage() {} func (x *CreateConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[32] + mi := &file_streaming_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2321,7 +2760,7 @@ func (x *CreateConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateConsumerRequest.ProtoReflect.Descriptor instead. func (*CreateConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{32} + return file_streaming_proto_rawDescGZIP(), []int{40} } func (x *CreateConsumerRequest) GetPchannel() *PChannelInfo { @@ -2342,7 +2781,7 @@ type CreateVChannelConsumersRequest struct { func (x *CreateVChannelConsumersRequest) Reset() { *x = CreateVChannelConsumersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[33] + mi := &file_streaming_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2355,7 +2794,7 @@ func (x *CreateVChannelConsumersRequest) String() string { func (*CreateVChannelConsumersRequest) ProtoMessage() {} func (x *CreateVChannelConsumersRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[33] + mi := &file_streaming_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2368,7 +2807,7 @@ func (x *CreateVChannelConsumersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumersRequest.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumersRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{33} + return file_streaming_proto_rawDescGZIP(), []int{41} } func (x *CreateVChannelConsumersRequest) GetCreateVchannels() []*CreateVChannelConsumerRequest { @@ -2393,7 +2832,7 @@ type CreateVChannelConsumerRequest struct { func (x *CreateVChannelConsumerRequest) Reset() { *x = CreateVChannelConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[34] + mi := &file_streaming_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2406,7 +2845,7 @@ func (x *CreateVChannelConsumerRequest) String() string { func (*CreateVChannelConsumerRequest) ProtoMessage() {} func (x *CreateVChannelConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[34] + mi := &file_streaming_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2419,7 +2858,7 @@ func (x *CreateVChannelConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumerRequest.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{34} + return file_streaming_proto_rawDescGZIP(), []int{42} } func (x *CreateVChannelConsumerRequest) GetVchannel() string { @@ -2455,7 +2894,7 @@ type CreateVChannelConsumersResponse struct { func (x *CreateVChannelConsumersResponse) Reset() { *x = CreateVChannelConsumersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[35] + mi := &file_streaming_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2468,7 +2907,7 @@ func (x *CreateVChannelConsumersResponse) String() string { func (*CreateVChannelConsumersResponse) ProtoMessage() {} func (x *CreateVChannelConsumersResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[35] + mi := &file_streaming_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2481,7 +2920,7 @@ func (x *CreateVChannelConsumersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumersResponse.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumersResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{35} + return file_streaming_proto_rawDescGZIP(), []int{43} } func (x *CreateVChannelConsumersResponse) GetCreateVchannels() []*CreateVChannelConsumerResponse { @@ -2508,7 +2947,7 @@ type CreateVChannelConsumerResponse struct { func (x *CreateVChannelConsumerResponse) Reset() { *x = CreateVChannelConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[36] + mi := &file_streaming_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2521,7 +2960,7 @@ func (x *CreateVChannelConsumerResponse) String() string { func (*CreateVChannelConsumerResponse) ProtoMessage() {} func (x *CreateVChannelConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[36] + mi := &file_streaming_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2534,7 +2973,7 @@ func (x *CreateVChannelConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateVChannelConsumerResponse.ProtoReflect.Descriptor instead. func (*CreateVChannelConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{36} + return file_streaming_proto_rawDescGZIP(), []int{44} } func (m *CreateVChannelConsumerResponse) GetResponse() isCreateVChannelConsumerResponse_Response { @@ -2586,7 +3025,7 @@ type CloseVChannelConsumerRequest struct { func (x *CloseVChannelConsumerRequest) Reset() { *x = CloseVChannelConsumerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[37] + mi := &file_streaming_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2599,7 +3038,7 @@ func (x *CloseVChannelConsumerRequest) String() string { func (*CloseVChannelConsumerRequest) ProtoMessage() {} func (x *CloseVChannelConsumerRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[37] + mi := &file_streaming_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2612,7 +3051,7 @@ func (x *CloseVChannelConsumerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseVChannelConsumerRequest.ProtoReflect.Descriptor instead. func (*CloseVChannelConsumerRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{37} + return file_streaming_proto_rawDescGZIP(), []int{45} } func (x *CloseVChannelConsumerRequest) GetConsumerId() int64 { @@ -2635,7 +3074,7 @@ type CloseVChannelConsumerResponse struct { func (x *CloseVChannelConsumerResponse) Reset() { *x = CloseVChannelConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[38] + mi := &file_streaming_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2648,7 +3087,7 @@ func (x *CloseVChannelConsumerResponse) String() string { func (*CloseVChannelConsumerResponse) ProtoMessage() {} func (x *CloseVChannelConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[38] + mi := &file_streaming_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2661,7 +3100,7 @@ func (x *CloseVChannelConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseVChannelConsumerResponse.ProtoReflect.Descriptor instead. func (*CloseVChannelConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{38} + return file_streaming_proto_rawDescGZIP(), []int{46} } func (x *CloseVChannelConsumerResponse) GetConsumerId() int64 { @@ -2691,7 +3130,7 @@ type ConsumeResponse struct { func (x *ConsumeResponse) Reset() { *x = ConsumeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[39] + mi := &file_streaming_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2704,7 +3143,7 @@ func (x *ConsumeResponse) String() string { func (*ConsumeResponse) ProtoMessage() {} func (x *ConsumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[39] + mi := &file_streaming_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2717,7 +3156,7 @@ func (x *ConsumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConsumeResponse.ProtoReflect.Descriptor instead. func (*ConsumeResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{39} + return file_streaming_proto_rawDescGZIP(), []int{47} } func (m *ConsumeResponse) GetResponse() isConsumeResponse_Response { @@ -2823,7 +3262,7 @@ type CreateConsumerResponse struct { func (x *CreateConsumerResponse) Reset() { *x = CreateConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[40] + mi := &file_streaming_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2836,7 +3275,7 @@ func (x *CreateConsumerResponse) String() string { func (*CreateConsumerResponse) ProtoMessage() {} func (x *CreateConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[40] + mi := &file_streaming_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2849,7 +3288,7 @@ func (x *CreateConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateConsumerResponse.ProtoReflect.Descriptor instead. func (*CreateConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{40} + return file_streaming_proto_rawDescGZIP(), []int{48} } func (x *CreateConsumerResponse) GetWalName() string { @@ -2878,7 +3317,7 @@ type ConsumeMessageReponse struct { func (x *ConsumeMessageReponse) Reset() { *x = ConsumeMessageReponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[41] + mi := &file_streaming_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2891,7 +3330,7 @@ func (x *ConsumeMessageReponse) String() string { func (*ConsumeMessageReponse) ProtoMessage() {} func (x *ConsumeMessageReponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[41] + mi := &file_streaming_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2904,7 +3343,7 @@ func (x *ConsumeMessageReponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConsumeMessageReponse.ProtoReflect.Descriptor instead. func (*ConsumeMessageReponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{41} + return file_streaming_proto_rawDescGZIP(), []int{49} } func (x *ConsumeMessageReponse) GetConsumerId() int64 { @@ -2930,7 +3369,7 @@ type CloseConsumerResponse struct { func (x *CloseConsumerResponse) Reset() { *x = CloseConsumerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[42] + mi := &file_streaming_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2943,7 +3382,7 @@ func (x *CloseConsumerResponse) String() string { func (*CloseConsumerResponse) ProtoMessage() {} func (x *CloseConsumerResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[42] + mi := &file_streaming_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2956,7 +3395,7 @@ func (x *CloseConsumerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseConsumerResponse.ProtoReflect.Descriptor instead. func (*CloseConsumerResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{42} + return file_streaming_proto_rawDescGZIP(), []int{50} } // StreamingManagerAssignRequest is the request message of Assign RPC. @@ -2971,7 +3410,7 @@ type StreamingNodeManagerAssignRequest struct { func (x *StreamingNodeManagerAssignRequest) Reset() { *x = StreamingNodeManagerAssignRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[43] + mi := &file_streaming_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2984,7 +3423,7 @@ func (x *StreamingNodeManagerAssignRequest) String() string { func (*StreamingNodeManagerAssignRequest) ProtoMessage() {} func (x *StreamingNodeManagerAssignRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[43] + mi := &file_streaming_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2997,7 +3436,7 @@ func (x *StreamingNodeManagerAssignRequest) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerAssignRequest.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerAssignRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{43} + return file_streaming_proto_rawDescGZIP(), []int{51} } func (x *StreamingNodeManagerAssignRequest) GetPchannel() *PChannelInfo { @@ -3016,7 +3455,7 @@ type StreamingNodeManagerAssignResponse struct { func (x *StreamingNodeManagerAssignResponse) Reset() { *x = StreamingNodeManagerAssignResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[44] + mi := &file_streaming_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3029,7 +3468,7 @@ func (x *StreamingNodeManagerAssignResponse) String() string { func (*StreamingNodeManagerAssignResponse) ProtoMessage() {} func (x *StreamingNodeManagerAssignResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[44] + mi := &file_streaming_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3042,7 +3481,7 @@ func (x *StreamingNodeManagerAssignResponse) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerAssignResponse.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerAssignResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{44} + return file_streaming_proto_rawDescGZIP(), []int{52} } type StreamingNodeManagerRemoveRequest struct { @@ -3056,7 +3495,7 @@ type StreamingNodeManagerRemoveRequest struct { func (x *StreamingNodeManagerRemoveRequest) Reset() { *x = StreamingNodeManagerRemoveRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[45] + mi := &file_streaming_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3069,7 +3508,7 @@ func (x *StreamingNodeManagerRemoveRequest) String() string { func (*StreamingNodeManagerRemoveRequest) ProtoMessage() {} func (x *StreamingNodeManagerRemoveRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[45] + mi := &file_streaming_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3082,7 +3521,7 @@ func (x *StreamingNodeManagerRemoveRequest) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerRemoveRequest.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerRemoveRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{45} + return file_streaming_proto_rawDescGZIP(), []int{53} } func (x *StreamingNodeManagerRemoveRequest) GetPchannel() *PChannelInfo { @@ -3101,7 +3540,7 @@ type StreamingNodeManagerRemoveResponse struct { func (x *StreamingNodeManagerRemoveResponse) Reset() { *x = StreamingNodeManagerRemoveResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[46] + mi := &file_streaming_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3114,7 +3553,7 @@ func (x *StreamingNodeManagerRemoveResponse) String() string { func (*StreamingNodeManagerRemoveResponse) ProtoMessage() {} func (x *StreamingNodeManagerRemoveResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[46] + mi := &file_streaming_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3127,7 +3566,7 @@ func (x *StreamingNodeManagerRemoveResponse) ProtoReflect() protoreflect.Message // Deprecated: Use StreamingNodeManagerRemoveResponse.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerRemoveResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{46} + return file_streaming_proto_rawDescGZIP(), []int{54} } type StreamingNodeManagerCollectStatusRequest struct { @@ -3139,7 +3578,7 @@ type StreamingNodeManagerCollectStatusRequest struct { func (x *StreamingNodeManagerCollectStatusRequest) Reset() { *x = StreamingNodeManagerCollectStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[47] + mi := &file_streaming_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3152,7 +3591,7 @@ func (x *StreamingNodeManagerCollectStatusRequest) String() string { func (*StreamingNodeManagerCollectStatusRequest) ProtoMessage() {} func (x *StreamingNodeManagerCollectStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[47] + mi := &file_streaming_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3165,7 +3604,7 @@ func (x *StreamingNodeManagerCollectStatusRequest) ProtoReflect() protoreflect.M // Deprecated: Use StreamingNodeManagerCollectStatusRequest.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerCollectStatusRequest) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{47} + return file_streaming_proto_rawDescGZIP(), []int{55} } type StreamingNodeBalanceAttributes struct { @@ -3177,7 +3616,7 @@ type StreamingNodeBalanceAttributes struct { func (x *StreamingNodeBalanceAttributes) Reset() { *x = StreamingNodeBalanceAttributes{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[48] + mi := &file_streaming_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3190,7 +3629,7 @@ func (x *StreamingNodeBalanceAttributes) String() string { func (*StreamingNodeBalanceAttributes) ProtoMessage() {} func (x *StreamingNodeBalanceAttributes) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[48] + mi := &file_streaming_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3203,7 +3642,7 @@ func (x *StreamingNodeBalanceAttributes) ProtoReflect() protoreflect.Message { // Deprecated: Use StreamingNodeBalanceAttributes.ProtoReflect.Descriptor instead. func (*StreamingNodeBalanceAttributes) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{48} + return file_streaming_proto_rawDescGZIP(), []int{56} } type StreamingNodeManagerCollectStatusResponse struct { @@ -3217,7 +3656,7 @@ type StreamingNodeManagerCollectStatusResponse struct { func (x *StreamingNodeManagerCollectStatusResponse) Reset() { *x = StreamingNodeManagerCollectStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[49] + mi := &file_streaming_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3230,7 +3669,7 @@ func (x *StreamingNodeManagerCollectStatusResponse) String() string { func (*StreamingNodeManagerCollectStatusResponse) ProtoMessage() {} func (x *StreamingNodeManagerCollectStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[49] + mi := &file_streaming_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3243,7 +3682,7 @@ func (x *StreamingNodeManagerCollectStatusResponse) ProtoReflect() protoreflect. // Deprecated: Use StreamingNodeManagerCollectStatusResponse.ProtoReflect.Descriptor instead. func (*StreamingNodeManagerCollectStatusResponse) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{49} + return file_streaming_proto_rawDescGZIP(), []int{57} } func (x *StreamingNodeManagerCollectStatusResponse) GetBalanceAttributes() *StreamingNodeBalanceAttributes { @@ -3277,7 +3716,7 @@ type SegmentAssignmentMeta struct { func (x *SegmentAssignmentMeta) Reset() { *x = SegmentAssignmentMeta{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[50] + mi := &file_streaming_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3290,7 +3729,7 @@ func (x *SegmentAssignmentMeta) String() string { func (*SegmentAssignmentMeta) ProtoMessage() {} func (x *SegmentAssignmentMeta) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[50] + mi := &file_streaming_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3303,7 +3742,7 @@ func (x *SegmentAssignmentMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentAssignmentMeta.ProtoReflect.Descriptor instead. func (*SegmentAssignmentMeta) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{50} + return file_streaming_proto_rawDescGZIP(), []int{58} } func (x *SegmentAssignmentMeta) GetCollectionId() int64 { @@ -3366,7 +3805,7 @@ type SegmentAssignmentStat struct { func (x *SegmentAssignmentStat) Reset() { *x = SegmentAssignmentStat{} if protoimpl.UnsafeEnabled { - mi := &file_streaming_proto_msgTypes[51] + mi := &file_streaming_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3379,7 +3818,7 @@ func (x *SegmentAssignmentStat) String() string { func (*SegmentAssignmentStat) ProtoMessage() {} func (x *SegmentAssignmentStat) ProtoReflect() protoreflect.Message { - mi := &file_streaming_proto_msgTypes[51] + mi := &file_streaming_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3392,7 +3831,7 @@ func (x *SegmentAssignmentStat) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentAssignmentStat.ProtoReflect.Descriptor instead. func (*SegmentAssignmentStat) Descriptor() ([]byte, []int) { - return file_streaming_proto_rawDescGZIP(), []int{51} + return file_streaming_proto_rawDescGZIP(), []int{59} } func (x *SegmentAssignmentStat) GetMaxBinarySize() uint64 { @@ -3486,578 +3925,648 @@ var file_streaming_proto_rawDesc = []byte{ 0x73, 0x22, 0x3b, 0x0a, 0x0b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x69, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x6f, 0x63, 0x61, - 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x22, 0xa4, + 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x22, 0xbf, 0x01, 0x0a, 0x0d, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, - 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 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, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x2a, 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, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x4c, 0x0a, 0x10, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, - 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 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, 0xd7, 0x01, 0x0a, 0x11, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x50, 0x0a, 0x07, 0x72, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6d, 0x69, 0x6c, + 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 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, 0x12, 0x40, 0x0a, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2a, 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, 0x54, 0x61, 0x73, 0x6b, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x32, 0x0a, 0x15, + 0x61, 0x63, 0x6b, 0x65, 0x64, 0x5f, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x62, + 0x69, 0x74, 0x6d, 0x61, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x13, 0x61, 0x63, 0x6b, + 0x65, 0x64, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x42, 0x69, 0x74, 0x6d, 0x61, 0x70, + 0x22, 0x4c, 0x0a, 0x10, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x01, 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, 0xfa, + 0x01, 0x0a, 0x11, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x50, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 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, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, + 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64, 0x1a, 0x70, 0x0a, 0x0c, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4a, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 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, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x1a, 0x70, 0x0a, 0x0c, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4a, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 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, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x54, 0x0a, 0x13, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, + 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, 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, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 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, + 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, 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, + 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, + 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, 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, + 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, 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, + 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, + 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, 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, 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, + 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, 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, + 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, 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, + 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, 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, + 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, 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, + 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, - 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, + 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, 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, + 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, 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, + 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, 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, + 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, 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, 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, + 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, 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, + 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, + 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, 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, 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, 0x77, 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, 0x2a, 0xdc, 0x03, 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, 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, + 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, 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, 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, 0x84, 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, + 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, 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, 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, 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, + 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, 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, 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, + 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, 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, 0x33, 0x5a, 0x31, 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, 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, + 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, 0x33, 0x5a, 0x31, 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, 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 ( @@ -4073,7 +4582,7 @@ func file_streaming_proto_rawDescGZIP() []byte { } var file_streaming_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 53) +var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 61) var file_streaming_proto_goTypes = []interface{}{ (PChannelMetaState)(0), // 0: milvus.proto.streaming.PChannelMetaState (BroadcastTaskState)(0), // 1: milvus.proto.streaming.BroadcastTaskState @@ -4086,147 +4595,166 @@ var file_streaming_proto_goTypes = []interface{}{ (*BroadcastTask)(nil), // 8: milvus.proto.streaming.BroadcastTask (*BroadcastRequest)(nil), // 9: milvus.proto.streaming.BroadcastRequest (*BroadcastResponse)(nil), // 10: milvus.proto.streaming.BroadcastResponse - (*AssignmentDiscoverRequest)(nil), // 11: milvus.proto.streaming.AssignmentDiscoverRequest - (*ReportAssignmentErrorRequest)(nil), // 12: milvus.proto.streaming.ReportAssignmentErrorRequest - (*CloseAssignmentDiscoverRequest)(nil), // 13: milvus.proto.streaming.CloseAssignmentDiscoverRequest - (*AssignmentDiscoverResponse)(nil), // 14: milvus.proto.streaming.AssignmentDiscoverResponse - (*FullStreamingNodeAssignmentWithVersion)(nil), // 15: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion - (*CloseAssignmentDiscoverResponse)(nil), // 16: milvus.proto.streaming.CloseAssignmentDiscoverResponse - (*StreamingNodeInfo)(nil), // 17: milvus.proto.streaming.StreamingNodeInfo - (*StreamingNodeAssignment)(nil), // 18: milvus.proto.streaming.StreamingNodeAssignment - (*DeliverPolicy)(nil), // 19: milvus.proto.streaming.DeliverPolicy - (*DeliverFilter)(nil), // 20: milvus.proto.streaming.DeliverFilter - (*DeliverFilterTimeTickGT)(nil), // 21: milvus.proto.streaming.DeliverFilterTimeTickGT - (*DeliverFilterTimeTickGTE)(nil), // 22: milvus.proto.streaming.DeliverFilterTimeTickGTE - (*DeliverFilterMessageType)(nil), // 23: milvus.proto.streaming.DeliverFilterMessageType - (*StreamingError)(nil), // 24: milvus.proto.streaming.StreamingError - (*ProduceRequest)(nil), // 25: milvus.proto.streaming.ProduceRequest - (*CreateProducerRequest)(nil), // 26: milvus.proto.streaming.CreateProducerRequest - (*ProduceMessageRequest)(nil), // 27: milvus.proto.streaming.ProduceMessageRequest - (*CloseProducerRequest)(nil), // 28: milvus.proto.streaming.CloseProducerRequest - (*ProduceResponse)(nil), // 29: milvus.proto.streaming.ProduceResponse - (*CreateProducerResponse)(nil), // 30: milvus.proto.streaming.CreateProducerResponse - (*ProduceMessageResponse)(nil), // 31: milvus.proto.streaming.ProduceMessageResponse - (*ProduceMessageResponseResult)(nil), // 32: milvus.proto.streaming.ProduceMessageResponseResult - (*CloseProducerResponse)(nil), // 33: milvus.proto.streaming.CloseProducerResponse - (*ConsumeRequest)(nil), // 34: milvus.proto.streaming.ConsumeRequest - (*CloseConsumerRequest)(nil), // 35: milvus.proto.streaming.CloseConsumerRequest - (*CreateConsumerRequest)(nil), // 36: milvus.proto.streaming.CreateConsumerRequest - (*CreateVChannelConsumersRequest)(nil), // 37: milvus.proto.streaming.CreateVChannelConsumersRequest - (*CreateVChannelConsumerRequest)(nil), // 38: milvus.proto.streaming.CreateVChannelConsumerRequest - (*CreateVChannelConsumersResponse)(nil), // 39: milvus.proto.streaming.CreateVChannelConsumersResponse - (*CreateVChannelConsumerResponse)(nil), // 40: milvus.proto.streaming.CreateVChannelConsumerResponse - (*CloseVChannelConsumerRequest)(nil), // 41: milvus.proto.streaming.CloseVChannelConsumerRequest - (*CloseVChannelConsumerResponse)(nil), // 42: milvus.proto.streaming.CloseVChannelConsumerResponse - (*ConsumeResponse)(nil), // 43: milvus.proto.streaming.ConsumeResponse - (*CreateConsumerResponse)(nil), // 44: milvus.proto.streaming.CreateConsumerResponse - (*ConsumeMessageReponse)(nil), // 45: milvus.proto.streaming.ConsumeMessageReponse - (*CloseConsumerResponse)(nil), // 46: milvus.proto.streaming.CloseConsumerResponse - (*StreamingNodeManagerAssignRequest)(nil), // 47: milvus.proto.streaming.StreamingNodeManagerAssignRequest - (*StreamingNodeManagerAssignResponse)(nil), // 48: milvus.proto.streaming.StreamingNodeManagerAssignResponse - (*StreamingNodeManagerRemoveRequest)(nil), // 49: milvus.proto.streaming.StreamingNodeManagerRemoveRequest - (*StreamingNodeManagerRemoveResponse)(nil), // 50: milvus.proto.streaming.StreamingNodeManagerRemoveResponse - (*StreamingNodeManagerCollectStatusRequest)(nil), // 51: milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest - (*StreamingNodeBalanceAttributes)(nil), // 52: milvus.proto.streaming.StreamingNodeBalanceAttributes - (*StreamingNodeManagerCollectStatusResponse)(nil), // 53: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse - (*SegmentAssignmentMeta)(nil), // 54: milvus.proto.streaming.SegmentAssignmentMeta - (*SegmentAssignmentStat)(nil), // 55: milvus.proto.streaming.SegmentAssignmentStat - nil, // 56: milvus.proto.streaming.BroadcastResponse.ResultsEntry - (*messagespb.Message)(nil), // 57: milvus.proto.messages.Message - (*emptypb.Empty)(nil), // 58: google.protobuf.Empty - (*messagespb.MessageID)(nil), // 59: milvus.proto.messages.MessageID - (messagespb.MessageType)(0), // 60: milvus.proto.messages.MessageType - (*messagespb.TxnContext)(nil), // 61: milvus.proto.messages.TxnContext - (*anypb.Any)(nil), // 62: google.protobuf.Any - (*messagespb.ImmutableMessage)(nil), // 63: milvus.proto.messages.ImmutableMessage - (*milvuspb.GetComponentStatesRequest)(nil), // 64: milvus.proto.milvus.GetComponentStatesRequest - (*milvuspb.ComponentStates)(nil), // 65: milvus.proto.milvus.ComponentStates + (*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 + nil, // 64: milvus.proto.streaming.BroadcastResponse.ResultsEntry + (*messagespb.Message)(nil), // 65: milvus.proto.messages.Message + (*messagespb.BroadcastEvent)(nil), // 66: milvus.proto.messages.BroadcastEvent + (*emptypb.Empty)(nil), // 67: google.protobuf.Empty + (*messagespb.MessageID)(nil), // 68: milvus.proto.messages.MessageID + (messagespb.MessageType)(0), // 69: milvus.proto.messages.MessageType + (*messagespb.TxnContext)(nil), // 70: milvus.proto.messages.TxnContext + (*anypb.Any)(nil), // 71: google.protobuf.Any + (*messagespb.ImmutableMessage)(nil), // 72: milvus.proto.messages.ImmutableMessage + (*milvuspb.GetComponentStatesRequest)(nil), // 73: milvus.proto.milvus.GetComponentStatesRequest + (*milvuspb.ComponentStates)(nil), // 74: milvus.proto.milvus.ComponentStates } var file_streaming_proto_depIdxs = []int32{ - 17, // 0: milvus.proto.streaming.PChannelAssignmentLog.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 25, // 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 - 17, // 2: milvus.proto.streaming.PChannelMeta.node:type_name -> milvus.proto.streaming.StreamingNodeInfo + 25, // 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 - 57, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message + 65, // 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 - 57, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message - 56, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry - 12, // 9: milvus.proto.streaming.AssignmentDiscoverRequest.report_error:type_name -> milvus.proto.streaming.ReportAssignmentErrorRequest - 13, // 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 - 24, // 12: milvus.proto.streaming.ReportAssignmentErrorRequest.err:type_name -> milvus.proto.streaming.StreamingError - 15, // 13: milvus.proto.streaming.AssignmentDiscoverResponse.full_assignment:type_name -> milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion - 16, // 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 - 18, // 16: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.assignments:type_name -> milvus.proto.streaming.StreamingNodeAssignment - 17, // 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 - 58, // 19: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty - 58, // 20: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty - 59, // 21: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID - 59, // 22: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID - 21, // 23: milvus.proto.streaming.DeliverFilter.time_tick_gt:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGT - 22, // 24: milvus.proto.streaming.DeliverFilter.time_tick_gte:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGTE - 23, // 25: milvus.proto.streaming.DeliverFilter.message_type:type_name -> milvus.proto.streaming.DeliverFilterMessageType - 60, // 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 - 27, // 28: milvus.proto.streaming.ProduceRequest.produce:type_name -> milvus.proto.streaming.ProduceMessageRequest - 28, // 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 - 57, // 31: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message - 30, // 32: milvus.proto.streaming.ProduceResponse.create:type_name -> milvus.proto.streaming.CreateProducerResponse - 31, // 33: milvus.proto.streaming.ProduceResponse.produce:type_name -> milvus.proto.streaming.ProduceMessageResponse - 33, // 34: milvus.proto.streaming.ProduceResponse.close:type_name -> milvus.proto.streaming.CloseProducerResponse - 32, // 35: milvus.proto.streaming.ProduceMessageResponse.result:type_name -> milvus.proto.streaming.ProduceMessageResponseResult - 24, // 36: milvus.proto.streaming.ProduceMessageResponse.error:type_name -> milvus.proto.streaming.StreamingError - 59, // 37: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID - 61, // 38: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext - 62, // 39: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any - 38, // 40: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumer:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest - 37, // 41: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumers:type_name -> milvus.proto.streaming.CreateVChannelConsumersRequest - 41, // 42: milvus.proto.streaming.ConsumeRequest.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerRequest - 35, // 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 - 38, // 45: milvus.proto.streaming.CreateVChannelConsumersRequest.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest - 19, // 46: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_policy:type_name -> milvus.proto.streaming.DeliverPolicy - 20, // 47: milvus.proto.streaming.CreateVChannelConsumerRequest.deliver_filters:type_name -> milvus.proto.streaming.DeliverFilter - 40, // 48: milvus.proto.streaming.CreateVChannelConsumersResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse - 24, // 49: milvus.proto.streaming.CreateVChannelConsumerResponse.error:type_name -> milvus.proto.streaming.StreamingError - 44, // 50: milvus.proto.streaming.ConsumeResponse.create:type_name -> milvus.proto.streaming.CreateConsumerResponse - 45, // 51: milvus.proto.streaming.ConsumeResponse.consume:type_name -> milvus.proto.streaming.ConsumeMessageReponse - 40, // 52: milvus.proto.streaming.ConsumeResponse.create_vchannel:type_name -> milvus.proto.streaming.CreateVChannelConsumerResponse - 39, // 53: milvus.proto.streaming.ConsumeResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumersResponse - 42, // 54: milvus.proto.streaming.ConsumeResponse.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerResponse - 46, // 55: milvus.proto.streaming.ConsumeResponse.close:type_name -> milvus.proto.streaming.CloseConsumerResponse - 63, // 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 - 52, // 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 - 55, // 61: milvus.proto.streaming.SegmentAssignmentMeta.stat:type_name -> milvus.proto.streaming.SegmentAssignmentStat - 32, // 62: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult - 64, // 63: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 9, // 64: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest - 11, // 65: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest - 25, // 66: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest - 34, // 67: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest - 47, // 68: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest - 49, // 69: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest - 51, // 70: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest - 65, // 71: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 10, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse - 14, // 73: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse - 29, // 74: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse - 43, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse - 48, // 76: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse - 50, // 77: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse - 53, // 78: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse - 71, // [71:79] is the sub-list for method output_type - 63, // [63:71] is the sub-list for method input_type - 63, // [63:63] is the sub-list for extension type_name - 63, // [63:63] is the sub-list for extension extendee - 0, // [0:63] is the sub-list for field type_name + 65, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message + 64, // 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 + 66, // 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 + 66, // 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 + 67, // 25: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty + 67, // 26: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty + 68, // 27: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID + 68, // 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 + 69, // 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 + 65, // 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 + 68, // 43: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID + 70, // 44: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext + 71, // 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 + 72, // 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 + 40, // 68: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 73, // 69: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 9, // 70: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest + 11, // 71: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:input_type -> milvus.proto.streaming.BroadcastAckRequest + 13, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:input_type -> milvus.proto.streaming.BroadcastWatchRequest + 19, // 73: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest + 33, // 74: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest + 42, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest + 55, // 76: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest + 57, // 77: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest + 59, // 78: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + 74, // 79: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 10, // 80: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse + 12, // 81: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:output_type -> milvus.proto.streaming.BroadcastAckResponse + 16, // 82: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:output_type -> milvus.proto.streaming.BroadcastWatchResponse + 22, // 83: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse + 37, // 84: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse + 51, // 85: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse + 56, // 86: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse + 58, // 87: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse + 61, // 88: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + 79, // [79:89] is the sub-list for method output_type + 69, // [69:79] is the sub-list for method input_type + 69, // [69:69] is the sub-list for extension type_name + 69, // [69:69] is the sub-list for extension extendee + 0, // [0:69] is the sub-list for field type_name } func init() { file_streaming_proto_init() } @@ -4320,7 +4848,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignmentDiscoverRequest); i { + switch v := v.(*BroadcastAckRequest); i { case 0: return &v.state case 1: @@ -4332,7 +4860,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReportAssignmentErrorRequest); i { + switch v := v.(*BroadcastAckResponse); i { case 0: return &v.state case 1: @@ -4344,7 +4872,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseAssignmentDiscoverRequest); i { + switch v := v.(*BroadcastWatchRequest); i { case 0: return &v.state case 1: @@ -4356,7 +4884,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignmentDiscoverResponse); i { + switch v := v.(*BroadcastCreateEventWatchRequest); i { case 0: return &v.state case 1: @@ -4368,7 +4896,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FullStreamingNodeAssignmentWithVersion); i { + switch v := v.(*CloseBroadcastWatchRequest); i { case 0: return &v.state case 1: @@ -4380,7 +4908,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseAssignmentDiscoverResponse); i { + switch v := v.(*BroadcastWatchResponse); i { case 0: return &v.state case 1: @@ -4392,7 +4920,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeInfo); i { + switch v := v.(*BroadcastEventWatchResponse); i { case 0: return &v.state case 1: @@ -4404,7 +4932,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeAssignment); i { + switch v := v.(*CloseBroadcastWatchResponse); i { case 0: return &v.state case 1: @@ -4416,7 +4944,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeliverPolicy); i { + switch v := v.(*AssignmentDiscoverRequest); i { case 0: return &v.state case 1: @@ -4428,7 +4956,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeliverFilter); i { + switch v := v.(*ReportAssignmentErrorRequest); i { case 0: return &v.state case 1: @@ -4440,7 +4968,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeliverFilterTimeTickGT); i { + switch v := v.(*CloseAssignmentDiscoverRequest); i { case 0: return &v.state case 1: @@ -4452,7 +4980,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeliverFilterTimeTickGTE); i { + switch v := v.(*AssignmentDiscoverResponse); i { case 0: return &v.state case 1: @@ -4464,7 +4992,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeliverFilterMessageType); i { + switch v := v.(*FullStreamingNodeAssignmentWithVersion); i { case 0: return &v.state case 1: @@ -4476,7 +5004,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingError); i { + switch v := v.(*CloseAssignmentDiscoverResponse); i { case 0: return &v.state case 1: @@ -4488,7 +5016,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProduceRequest); i { + switch v := v.(*StreamingNodeInfo); i { case 0: return &v.state case 1: @@ -4500,7 +5028,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateProducerRequest); i { + switch v := v.(*StreamingNodeAssignment); i { case 0: return &v.state case 1: @@ -4512,7 +5040,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProduceMessageRequest); i { + switch v := v.(*DeliverPolicy); i { case 0: return &v.state case 1: @@ -4524,7 +5052,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseProducerRequest); i { + switch v := v.(*DeliverFilter); i { case 0: return &v.state case 1: @@ -4536,7 +5064,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProduceResponse); i { + switch v := v.(*DeliverFilterTimeTickGT); i { case 0: return &v.state case 1: @@ -4548,7 +5076,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateProducerResponse); i { + switch v := v.(*DeliverFilterTimeTickGTE); i { case 0: return &v.state case 1: @@ -4560,7 +5088,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProduceMessageResponse); i { + switch v := v.(*DeliverFilterMessageType); i { case 0: return &v.state case 1: @@ -4572,7 +5100,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProduceMessageResponseResult); i { + switch v := v.(*StreamingError); i { case 0: return &v.state case 1: @@ -4584,7 +5112,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseProducerResponse); i { + switch v := v.(*ProduceRequest); i { case 0: return &v.state case 1: @@ -4596,7 +5124,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConsumeRequest); i { + switch v := v.(*CreateProducerRequest); i { case 0: return &v.state case 1: @@ -4608,7 +5136,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseConsumerRequest); i { + switch v := v.(*ProduceMessageRequest); i { case 0: return &v.state case 1: @@ -4620,7 +5148,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateConsumerRequest); i { + switch v := v.(*CloseProducerRequest); i { case 0: return &v.state case 1: @@ -4632,7 +5160,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateVChannelConsumersRequest); i { + switch v := v.(*ProduceResponse); i { case 0: return &v.state case 1: @@ -4644,7 +5172,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateVChannelConsumerRequest); i { + switch v := v.(*CreateProducerResponse); i { case 0: return &v.state case 1: @@ -4656,7 +5184,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateVChannelConsumersResponse); i { + switch v := v.(*ProduceMessageResponse); i { case 0: return &v.state case 1: @@ -4668,7 +5196,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateVChannelConsumerResponse); i { + switch v := v.(*ProduceMessageResponseResult); i { case 0: return &v.state case 1: @@ -4680,7 +5208,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseVChannelConsumerRequest); i { + switch v := v.(*CloseProducerResponse); i { case 0: return &v.state case 1: @@ -4692,7 +5220,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseVChannelConsumerResponse); i { + switch v := v.(*ConsumeRequest); i { case 0: return &v.state case 1: @@ -4704,7 +5232,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConsumeResponse); i { + switch v := v.(*CloseConsumerRequest); i { case 0: return &v.state case 1: @@ -4716,7 +5244,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateConsumerResponse); i { + switch v := v.(*CreateConsumerRequest); i { case 0: return &v.state case 1: @@ -4728,7 +5256,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConsumeMessageReponse); i { + switch v := v.(*CreateVChannelConsumersRequest); i { case 0: return &v.state case 1: @@ -4740,7 +5268,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseConsumerResponse); i { + switch v := v.(*CreateVChannelConsumerRequest); i { case 0: return &v.state case 1: @@ -4752,7 +5280,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeManagerAssignRequest); i { + switch v := v.(*CreateVChannelConsumersResponse); i { case 0: return &v.state case 1: @@ -4764,7 +5292,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeManagerAssignResponse); i { + switch v := v.(*CreateVChannelConsumerResponse); i { case 0: return &v.state case 1: @@ -4776,7 +5304,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeManagerRemoveRequest); i { + switch v := v.(*CloseVChannelConsumerRequest); i { case 0: return &v.state case 1: @@ -4788,7 +5316,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeManagerRemoveResponse); i { + switch v := v.(*CloseVChannelConsumerResponse); i { case 0: return &v.state case 1: @@ -4800,7 +5328,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeManagerCollectStatusRequest); i { + switch v := v.(*ConsumeResponse); i { case 0: return &v.state case 1: @@ -4812,7 +5340,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeBalanceAttributes); i { + switch v := v.(*CreateConsumerResponse); i { case 0: return &v.state case 1: @@ -4824,7 +5352,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StreamingNodeManagerCollectStatusResponse); i { + switch v := v.(*ConsumeMessageReponse); i { case 0: return &v.state case 1: @@ -4836,7 +5364,7 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SegmentAssignmentMeta); i { + switch v := v.(*CloseConsumerResponse); i { case 0: return &v.state case 1: @@ -4848,6 +5376,102 @@ func file_streaming_proto_init() { } } file_streaming_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerAssignRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerAssignResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerRemoveRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerRemoveResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerCollectStatusRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeBalanceAttributes); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StreamingNodeManagerCollectStatusResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SegmentAssignmentMeta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_streaming_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SegmentAssignmentStat); i { case 0: return &v.state @@ -4860,49 +5484,57 @@ func file_streaming_proto_init() { } } } - file_streaming_proto_msgTypes[7].OneofWrappers = []interface{}{ + 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[10].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[18].OneofWrappers = []interface{}{ (*AssignmentDiscoverResponse_FullAssignment)(nil), (*AssignmentDiscoverResponse_Close)(nil), } - file_streaming_proto_msgTypes[15].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[23].OneofWrappers = []interface{}{ (*DeliverPolicy_All)(nil), (*DeliverPolicy_Latest)(nil), (*DeliverPolicy_StartFrom)(nil), (*DeliverPolicy_StartAfter)(nil), } - file_streaming_proto_msgTypes[16].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[24].OneofWrappers = []interface{}{ (*DeliverFilter_TimeTickGt)(nil), (*DeliverFilter_TimeTickGte)(nil), (*DeliverFilter_MessageType)(nil), } - file_streaming_proto_msgTypes[21].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[29].OneofWrappers = []interface{}{ (*ProduceRequest_Produce)(nil), (*ProduceRequest_Close)(nil), } - file_streaming_proto_msgTypes[25].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[33].OneofWrappers = []interface{}{ (*ProduceResponse_Create)(nil), (*ProduceResponse_Produce)(nil), (*ProduceResponse_Close)(nil), } - file_streaming_proto_msgTypes[27].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[35].OneofWrappers = []interface{}{ (*ProduceMessageResponse_Result)(nil), (*ProduceMessageResponse_Error)(nil), } - file_streaming_proto_msgTypes[30].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[38].OneofWrappers = []interface{}{ (*ConsumeRequest_CreateVchannelConsumer)(nil), (*ConsumeRequest_CreateVchannelConsumers)(nil), (*ConsumeRequest_CloseVchannel)(nil), (*ConsumeRequest_Close)(nil), } - file_streaming_proto_msgTypes[36].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[44].OneofWrappers = []interface{}{ (*CreateVChannelConsumerResponse_ConsumerId)(nil), (*CreateVChannelConsumerResponse_Error)(nil), } - file_streaming_proto_msgTypes[39].OneofWrappers = []interface{}{ + file_streaming_proto_msgTypes[47].OneofWrappers = []interface{}{ (*ConsumeResponse_Create)(nil), (*ConsumeResponse_Consume)(nil), (*ConsumeResponse_CreateVchannel)(nil), @@ -4916,7 +5548,7 @@ func file_streaming_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_streaming_proto_rawDesc, NumEnums: 4, - NumMessages: 53, + NumMessages: 61, NumExtensions: 0, NumServices: 5, }, diff --git a/pkg/proto/streamingpb/streaming_grpc.pb.go b/pkg/proto/streamingpb/streaming_grpc.pb.go index 1d3d8beef70f4..6540e92a20951 100644 --- a/pkg/proto/streamingpb/streaming_grpc.pb.go +++ b/pkg/proto/streamingpb/streaming_grpc.pb.go @@ -109,6 +109,8 @@ 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. @@ -118,6 +120,10 @@ type StreamingCoordBroadcastServiceClient interface { // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. // It performs an atomic broadcast to all wal, achieve eventual consistency. 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 { @@ -137,6 +143,46 @@ func (c *streamingCoordBroadcastServiceClient) Broadcast(ctx context.Context, in return out, nil } +func (c *streamingCoordBroadcastServiceClient) Ack(ctx context.Context, in *BroadcastAckRequest, opts ...grpc.CallOption) (*BroadcastAckResponse, error) { + out := new(BroadcastAckResponse) + err := c.cc.Invoke(ctx, StreamingCoordBroadcastService_Ack_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + 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 @@ -144,6 +190,10 @@ type StreamingCoordBroadcastServiceServer interface { // Broadcast receives broadcast messages from other component and make sure that the message is broadcast to all wal. // It performs an atomic broadcast to all wal, achieve eventual consistency. 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. @@ -153,6 +203,12 @@ type UnimplementedStreamingCoordBroadcastServiceServer struct { func (UnimplementedStreamingCoordBroadcastServiceServer) Broadcast(context.Context, *BroadcastRequest) (*BroadcastResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Broadcast not implemented") } +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 @@ -183,6 +239,50 @@ func _StreamingCoordBroadcastService_Broadcast_Handler(srv interface{}, ctx cont return interceptor(ctx, in, info, handler) } +func _StreamingCoordBroadcastService_Ack_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(BroadcastAckRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(StreamingCoordBroadcastServiceServer).Ack(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: StreamingCoordBroadcastService_Ack_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(StreamingCoordBroadcastServiceServer).Ack(ctx, req.(*BroadcastAckRequest)) + } + 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) @@ -194,8 +294,19 @@ var StreamingCoordBroadcastService_ServiceDesc = grpc.ServiceDesc{ MethodName: "Broadcast", Handler: _StreamingCoordBroadcastService_Broadcast_Handler, }, + { + MethodName: "Ack", + Handler: _StreamingCoordBroadcastService_Ack_Handler, + }, + }, + Streams: []grpc.StreamDesc{ + { + StreamName: "Watch", + Handler: _StreamingCoordBroadcastService_Watch_Handler, + ServerStreams: true, + ClientStreams: true, + }, }, - Streams: []grpc.StreamDesc{}, Metadata: "streaming.proto", } diff --git a/pkg/streaming/util/message/broadcast.go b/pkg/streaming/util/message/broadcast.go new file mode 100644 index 0000000000000..b29c94407a44e --- /dev/null +++ b/pkg/streaming/util/message/broadcast.go @@ -0,0 +1,111 @@ +package message + +import ( + "strconv" + + "google.golang.org/protobuf/proto" + + "github.com/milvus-io/milvus/pkg/proto/messagespb" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +// newBroadcastHeaderFromProto creates a BroadcastHeader from proto. +func newBroadcastHeaderFromProto(proto *messagespb.BroadcastHeader) *BroadcastHeader { + rks := make(typeutil.Set[ResourceKey], len(proto.ResourceKeys)) + for _, key := range proto.ResourceKeys { + rks.Insert(NewResourceKeyFromProto(key)) + } + return &BroadcastHeader{ + BroadcastID: proto.BroadcastId, + VChannels: proto.Vchannels, + ResourceKeys: rks, + } +} + +type BroadcastHeader struct { + BroadcastID uint64 + VChannels []string + ResourceKeys typeutil.Set[ResourceKey] +} + +// NewResourceKeyFromProto creates a ResourceKey from proto. +func NewResourceKeyFromProto(proto *messagespb.ResourceKey) ResourceKey { + return ResourceKey{ + Domain: proto.Domain, + Key: proto.Key, + } +} + +// newProtoFromResourceKey creates a set of proto from ResourceKey. +func newProtoFromResourceKey(keys ...ResourceKey) []*messagespb.ResourceKey { + deduplicated := typeutil.NewSet(keys...) + protos := make([]*messagespb.ResourceKey, 0, len(keys)) + for key := range deduplicated { + protos = append(protos, &messagespb.ResourceKey{ + Domain: key.Domain, + Key: key.Key, + }) + } + return protos +} + +type ResourceKey struct { + Domain messagespb.ResourceDomain + Key string +} + +func (rk *ResourceKey) IntoResourceKey() *messagespb.ResourceKey { + return &messagespb.ResourceKey{ + Domain: rk.Domain, + Key: rk.Key, + } +} + +// NewImportJobIDResourceKey creates a key for import job resource. +func NewImportJobIDResourceKey(importJobID int64) ResourceKey { + return ResourceKey{ + Domain: messagespb.ResourceDomain_ResourceDomainImportJobID, + Key: strconv.FormatInt(importJobID, 10), + } +} + +// NewCollectionNameResourceKey creates a key for collection name resource. +func NewCollectionNameResourceKey(collectionName string) ResourceKey { + return ResourceKey{ + Domain: messagespb.ResourceDomain_ResourceDomainCollectionName, + Key: collectionName, + } +} + +type BroadcastEvent = messagespb.BroadcastEvent + +// UniqueKeyOfBroadcastEvent returns a unique key for a broadcast event. +func UniqueKeyOfBroadcastEvent(ev *BroadcastEvent) string { + s, err := proto.Marshal(ev) + if err != nil { + panic(err) + } + return string(s) +} + +// NewResourceKeyAckOneBroadcastEvent creates a broadcast event for acking one key. +func NewResourceKeyAckOneBroadcastEvent(rk ResourceKey) *BroadcastEvent { + return &BroadcastEvent{ + Event: &messagespb.BroadcastEvent_ResourceKeyAckOne{ + ResourceKeyAckOne: &messagespb.BroadcastResourceKeyAckOne{ + ResourceKey: rk.IntoResourceKey(), + }, + }, + } +} + +// NewResourceKeyAckAllBroadcastEvent creates a broadcast event for ack all vchannel. +func NewResourceKeyAckAllBroadcastEvent(rk ResourceKey) *BroadcastEvent { + return &BroadcastEvent{ + Event: &messagespb.BroadcastEvent_ResourceKeyAckAll{ + ResourceKeyAckAll: &messagespb.BroadcastResourceKeyAckAll{ + ResourceKey: rk.IntoResourceKey(), + }, + }, + } +} diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index 527add53795d8..1668e4abd1e06 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -12,26 +12,26 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) -// NewMutableMessage creates a new mutable message. +// NewMutableMessageBeforeAppend creates a new mutable message. // !!! Only used at server side for streamingnode internal service, don't use it at client side. -func NewMutableMessage(payload []byte, properties map[string]string) MutableMessage { +func NewMutableMessageBeforeAppend(payload []byte, properties map[string]string) MutableMessage { m := &messageImpl{ payload: payload, properties: properties, } - // make a assertion by vchannel function. - m.assertNotBroadcast() return m } -// NewBroadcastMutableMessage creates a new broadcast mutable message. +// NewBroadcastMutableMessageBeforeAppend creates a new broadcast mutable message. // !!! Only used at server side for streamingcoord internal service, don't use it at client side. -func NewBroadcastMutableMessage(payload []byte, properties map[string]string) BroadcastMutableMessage { +func NewBroadcastMutableMessageBeforeAppend(payload []byte, properties map[string]string) BroadcastMutableMessage { m := &messageImpl{ payload: payload, properties: properties, } - m.assertBroadcast() + if !m.properties.Exist(messageBroadcastHeader) { + panic("current message is not a broadcast message") + } return m } @@ -126,7 +126,7 @@ func (b *mutableMesasgeBuilder[H, B]) WithVChannel(vchannel string) *mutableMesa } // WithBroadcast creates a new builder with broadcast property. -func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string) *mutableMesasgeBuilder[H, B] { +func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string, resourceKeys ...ResourceKey) *mutableMesasgeBuilder[H, B] { if len(vchannels) < 1 { panic("broadcast message must have at least one vchannel") } @@ -137,19 +137,21 @@ func (b *mutableMesasgeBuilder[H, B]) WithBroadcast(vchannels []string) *mutable panic("a broadcast message cannot set up vchannel property") } deduplicated := typeutil.NewSet(vchannels...) - vcs, err := EncodeProto(&messagespb.VChannels{ - Vchannels: deduplicated.Collect(), + + bh, err := EncodeProto(&messagespb.BroadcastHeader{ + Vchannels: deduplicated.Collect(), + ResourceKeys: newProtoFromResourceKey(resourceKeys...), }) if err != nil { panic("failed to encode vchannels") } - b.properties.Set(messageVChannels, vcs) + b.properties.Set(messageBroadcastHeader, bh) return b } // WithAllVChannel creates a new builder with all vchannel property. func (b *mutableMesasgeBuilder[H, B]) WithAllVChannel() *mutableMesasgeBuilder[H, B] { - if b.properties.Exist(messageVChannel) || b.properties.Exist(messageVChannels) { + if b.properties.Exist(messageVChannel) || b.properties.Exist(messageBroadcastHeader) { panic("a vchannel or broadcast message cannot set up all vchannel property") } b.allVChannel = true @@ -191,7 +193,7 @@ func (b *mutableMesasgeBuilder[H, B]) BuildMutable() (MutableMessage, error) { // Panic if not set payload and message type. // should only used at client side. func (b *mutableMesasgeBuilder[H, B]) BuildBroadcast() (BroadcastMutableMessage, error) { - if !b.properties.Exist(messageVChannels) { + if !b.properties.Exist(messageBroadcastHeader) { panic("a broadcast message builder not ready for vchannel field") } diff --git a/pkg/streaming/util/message/message.go b/pkg/streaming/util/message/message.go index 49a7361c82fcd..f8d3999ab08ba 100644 --- a/pkg/streaming/util/message/message.go +++ b/pkg/streaming/util/message/message.go @@ -1,6 +1,8 @@ package message -import "google.golang.org/protobuf/proto" +import ( + "google.golang.org/protobuf/proto" +) var ( _ BasicMessage = (*messageImpl)(nil) @@ -39,7 +41,12 @@ type BasicMessage interface { BarrierTimeTick() uint64 // TxnContext returns the transaction context of current message. + // If the message is not a transaction message, it will return nil. TxnContext() *TxnContext + + // BroadcastHeader returns the broadcast common header of the message. + // If the message is not a broadcast message, it will return 0. + BroadcastHeader() *BroadcastHeader } // MutableMessage is the mutable message interface. @@ -87,11 +94,11 @@ type MutableMessage interface { type BroadcastMutableMessage interface { BasicMessage - // BroadcastVChannels returns the target vchannels of the message broadcast. - // Those vchannels can be on multi pchannels. - BroadcastVChannels() []string + // WithBroadcastID sets the broadcast id of the message. + WithBroadcastID(broadcastID uint64) BroadcastMutableMessage // SplitIntoMutableMessage splits the broadcast message into multiple mutable messages. + // The broadcast id will be set into the properties of each message. SplitIntoMutableMessage() []MutableMessage } diff --git a/pkg/streaming/util/message/message_impl.go b/pkg/streaming/util/message/message_impl.go index 25c9bcad9ea8d..981edb6de3d4d 100644 --- a/pkg/streaming/util/message/message_impl.go +++ b/pkg/streaming/util/message/message_impl.go @@ -93,6 +93,24 @@ func (m *messageImpl) WithTxnContext(txnCtx TxnContext) MutableMessage { return m } +// WithBroadcastID sets the broadcast id of current message. +func (m *messageImpl) WithBroadcastID(id uint64) BroadcastMutableMessage { + bh := m.broadcastHeader() + if bh == nil { + panic("there's a bug in the message codes, broadcast header lost in properties of broadcast message") + } + if bh.BroadcastId != 0 { + panic("broadcast id already set in properties of broadcast message") + } + bh.BroadcastId = id + bhVal, err := EncodeProto(bh) + if err != nil { + panic("should not happen on broadcast header proto") + } + m.properties.Set(messageBroadcastHeader, bhVal) + return m +} + // IntoImmutableMessage converts current message to immutable message. func (m *messageImpl) IntoImmutableMessage(id MessageID) ImmutableMessage { return &immutableMessageImpl{ @@ -144,8 +162,10 @@ func (m *messageImpl) BarrierTimeTick() uint64 { // If the message is a all channel message, it will return "". // If the message is a broadcast message, it will panic. func (m *messageImpl) VChannel() string { - m.assertNotBroadcast() - + if m.properties.Exist(messageBroadcastHeader) && !m.properties.Exist(messageVChannel) { + // If a message is a broadcast message, it must have a vchannel properties in it after split. + panic("there's a bug in the message codes, vchannel lost in properties of broadcast message") + } value, ok := m.properties.Get(messageVChannel) if !ok { return "" @@ -153,22 +173,38 @@ func (m *messageImpl) VChannel() string { return value } -// BroadcastVChannels returns the vchannels of current message that want to broadcast. -// If the message is not a broadcast message, it will panic. -func (m *messageImpl) BroadcastVChannels() []string { - m.assertBroadcast() +// BroadcastHeader returns the broadcast header of current message. +func (m *messageImpl) BroadcastHeader() *BroadcastHeader { + header := m.broadcastHeader() + return newBroadcastHeaderFromProto(header) +} - value, _ := m.properties.Get(messageVChannels) - vcs := &messagespb.VChannels{} - if err := DecodeProto(value, vcs); err != nil { - panic("can not decode vchannels") +// broadcastHeader returns the broadcast header of current message. +func (m *messageImpl) broadcastHeader() *messagespb.BroadcastHeader { + value, ok := m.properties.Get(messageBroadcastHeader) + if !ok { + return nil + } + header := &messagespb.BroadcastHeader{} + if err := DecodeProto(value, header); err != nil { + panic("can not decode broadcast header") } - return vcs.Vchannels + return header } // SplitIntoMutableMessage splits the current broadcast message into multiple messages. func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage { - vchannels := m.BroadcastVChannels() + bh := m.broadcastHeader() + if bh == nil { + panic("there's a bug in the message codes, broadcast header lost in properties of broadcast message") + } + if len(bh.Vchannels) == 0 { + panic("there's a bug in the message codes, no vchannel in broadcast message") + } + if bh.BroadcastId == 0 { + panic("there's a bug in the message codes, no broadcast id in broadcast message") + } + vchannels := bh.Vchannels vchannelExist := make(map[string]struct{}, len(vchannels)) msgs := make([]MutableMessage, 0, len(vchannels)) @@ -178,9 +214,7 @@ func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage { newProperties := make(propertiesImpl, len(m.properties)) for key, val := range m.properties { - if key != messageVChannels { - newProperties.Set(key, val) - } + newProperties.Set(key, val) } newProperties.Set(messageVChannel, vchannel) if _, ok := vchannelExist[vchannel]; ok { @@ -195,18 +229,6 @@ func (m *messageImpl) SplitIntoMutableMessage() []MutableMessage { return msgs } -func (m *messageImpl) assertNotBroadcast() { - if m.properties.Exist(messageVChannels) { - panic("current message is a broadcast message") - } -} - -func (m *messageImpl) assertBroadcast() { - if !m.properties.Exist(messageVChannels) { - panic("current message is not a broadcast message") - } -} - type immutableMessageImpl struct { messageImpl id MessageID diff --git a/pkg/streaming/util/message/properties.go b/pkg/streaming/util/message/properties.go index 3f0d120e32fd4..de2268bed661f 100644 --- a/pkg/streaming/util/message/properties.go +++ b/pkg/streaming/util/message/properties.go @@ -10,7 +10,7 @@ const ( messageLastConfirmed = "_lc" // message last confirmed message id. messageLastConfirmedIDSameWithMessageID = "_lcs" // message last confirmed message id is the same with message id. messageVChannel = "_vc" // message virtual channel. - messageVChannels = "_vcs" // message virtual channels for broadcast message. + messageBroadcastHeader = "_bh" // message broadcast header. messageHeader = "_h" // specialized message header. messageTxnContext = "_tx" // transaction context. ) diff --git a/pkg/streaming/util/types/responses.go b/pkg/streaming/util/types/responses.go index 7f9dcf6908f89..9ee07f90a4892 100644 --- a/pkg/streaming/util/types/responses.go +++ b/pkg/streaming/util/types/responses.go @@ -9,8 +9,15 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/util/message" ) +type BroadcastAckRequest struct { + // BroadcastID is the broadcast id of the ack request. + BroadcastID uint64 + VChannel string +} + // BroadcastAppendResult is the result of broadcast append operation. type BroadcastAppendResult struct { + BroadcastID uint64 // the broadcast id of the append operation. AppendResults map[string]*AppendResult // make the channel name to the append result. } diff --git a/tests/integration/coorddownsearch/search_after_coord_down_test.go b/tests/integration/coorddownsearch/search_after_coord_down_test.go index ead15e6f19aa5..5b597760d79cf 100644 --- a/tests/integration/coorddownsearch/search_after_coord_down_test.go +++ b/tests/integration/coorddownsearch/search_after_coord_down_test.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" @@ -304,6 +305,7 @@ func (s *CoordDownSearch) searchAfterCoordDown() float64 { log.Info(fmt.Sprintf("=========================Failed search cost: %fs=========================", time.Since(failedStart).Seconds())) registry.ResetRegistration() + coordclient.ResetRegistration() log.Info("=========================restart Root Coordinators=========================") c.StartRootCoord() diff --git a/tests/integration/coordrecovery/coord_recovery_test.go b/tests/integration/coordrecovery/coord_recovery_test.go index b80c63023af71..65d922ea86831 100644 --- a/tests/integration/coordrecovery/coord_recovery_test.go +++ b/tests/integration/coordrecovery/coord_recovery_test.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" @@ -244,6 +245,7 @@ func (s *CoordSwitchSuite) switchCoord() float64 { start = time.Now() registry.ResetRegistration() + coordclient.ResetRegistration() c.StartRootCoord() log.Info("=========================RootCoord restarted=========================") diff --git a/tests/integration/minicluster_v2.go b/tests/integration/minicluster_v2.go index 3cde9a60bb51d..dda759a041e0b 100644 --- a/tests/integration/minicluster_v2.go +++ b/tests/integration/minicluster_v2.go @@ -37,6 +37,7 @@ import ( "google.golang.org/grpc/keepalive" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/coordinator/coordclient" grpcdatacoord "github.com/milvus-io/milvus/internal/distributed/datacoord" grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord/client" grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode" @@ -54,6 +55,7 @@ import ( "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/distributed/streamingnode" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/hookutil" @@ -77,6 +79,7 @@ func DefaultParams() map[string]string { // Notice: don't use ParamItem.Key here, the config key will be empty before param table init configMap = map[string]string{ + "mq.type": "rocksmq", "etcd.rootPath": testPath, "minio.rootPath": testPath, "localStorage.path": path.Join("/tmp", testPath), @@ -151,6 +154,8 @@ func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2, for k, v := range cluster.params { params.Save(k, v) } + paramtable.SetRole(typeutil.StandaloneRole) + // setup etcd client etcdConfig := ¶mtable.Get().EtcdCfg etcdCli, err := etcd.GetEtcdClient( @@ -166,9 +171,9 @@ func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2, } cluster.EtcdCli = etcdCli - if streamingutil.IsStreamingServiceEnabled() { - streaming.Init() - } + coordclient.ResetRegistration() + registry.ResetRegistration() + streaming.Init() cluster.MetaWatcher = &EtcdMetaWatcher{ rootPath: etcdConfig.RootPath.GetValue(), @@ -389,6 +394,7 @@ func (cluster *MiniClusterV2) StopRootCoord() { func (cluster *MiniClusterV2) StartRootCoord() { if cluster.RootCoord == nil { + coordclient.ResetRootCoordRegistration() var err error if cluster.RootCoord, err = grpcrootcoord.NewServer(cluster.ctx, cluster.factory); err != nil { panic(err) @@ -406,6 +412,7 @@ func (cluster *MiniClusterV2) StopDataCoord() { func (cluster *MiniClusterV2) StartDataCoord() { if cluster.DataCoord == nil { + coordclient.ResetRootCoordRegistration() var err error if cluster.DataCoord, err = grpcdatacoord.NewServer(cluster.ctx, cluster.factory); err != nil { panic(err) @@ -423,6 +430,7 @@ func (cluster *MiniClusterV2) StopQueryCoord() { func (cluster *MiniClusterV2) StartQueryCoord() { if cluster.QueryCoord == nil { + coordclient.ResetQueryCoordRegistration() var err error if cluster.QueryCoord, err = grpcquerycoord.NewServer(cluster.ctx, cluster.factory); err != nil { panic(err) @@ -486,10 +494,6 @@ func (cluster *MiniClusterV2) Stop() error { cluster.StopAllStreamingNodes() cluster.StopAllQueryNodes() - if streamingutil.IsStreamingServiceEnabled() { - streaming.Release() - } - cluster.IndexNode.Stop() log.Info("mini cluster indexNode stopped") @@ -505,6 +509,7 @@ func (cluster *MiniClusterV2) Stop() error { } } cluster.ChunkManager.RemoveWithPrefix(cluster.ctx, cluster.ChunkManager.RootPath()) + streaming.Release() return nil } diff --git a/tests/integration/suite.go b/tests/integration/suite.go index 884bf945b741a..d1bd7fc3e8ee7 100644 --- a/tests/integration/suite.go +++ b/tests/integration/suite.go @@ -30,12 +30,10 @@ import ( "go.uber.org/zap/zapcore" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" - "github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" - "github.com/milvus-io/milvus/pkg/util/paramtable" ) var caseTimeout time.Duration @@ -110,8 +108,6 @@ func (s *MiniClusterSuite) SetupTest() { // setup env value to init etcd source s.T().Setenv("etcd.endpoints", val) - params = paramtable.Get() - s.T().Log("Setup case timeout", caseTimeout) ctx, cancel := context.WithTimeout(context.Background(), caseTimeout) s.cancelFunc = cancel @@ -170,5 +166,4 @@ func (s *MiniClusterSuite) TearDownTest() { if s.Cluster != nil { s.Cluster.Stop() } - registry.ResetRegistration() }