aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--weed/filer_client/filer_client_accessor.go65
-rw-r--r--weed/mq/broker/broker_connect.go4
-rw-r--r--weed/mq/broker/broker_grpc_assign.go8
-rw-r--r--weed/mq/broker/broker_grpc_balance.go4
-rw-r--r--weed/mq/broker/broker_grpc_configure.go10
-rw-r--r--weed/mq/broker/broker_grpc_lookup.go4
-rw-r--r--weed/mq/broker/broker_grpc_pub_balancer.go11
-rw-r--r--weed/mq/broker/broker_grpc_sub.go131
-rw-r--r--weed/mq/broker/broker_grpc_sub_coordinator.go32
-rw-r--r--weed/mq/broker/broker_grpc_sub_follow.go101
-rw-r--r--weed/mq/broker/broker_server.go29
-rw-r--r--weed/mq/broker/broker_topic_conf_read_write.go54
-rw-r--r--weed/mq/broker/broker_topic_partition_read_write.go5
-rw-r--r--weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go21
-rw-r--r--weed/mq/client/cmd/weed_sub_record/subscriber_record.go22
-rw-r--r--weed/mq/client/pub_client/publish.go1
-rw-r--r--weed/mq/client/pub_client/scheduler.go10
-rw-r--r--weed/mq/client/sub_client/connect_to_sub_coordinator.go118
-rw-r--r--weed/mq/client/sub_client/on_each_partition.go125
-rw-r--r--weed/mq/client/sub_client/subscribe.go110
-rw-r--r--weed/mq/client/sub_client/subscriber.go46
-rw-r--r--weed/mq/pub_balancer/allocate.go62
-rw-r--r--weed/mq/pub_balancer/balance.go4
-rw-r--r--weed/mq/pub_balancer/balance_action.go6
-rw-r--r--weed/mq/pub_balancer/lookup.go2
-rw-r--r--weed/mq/pub_balancer/partition_list_broker.go16
-rw-r--r--weed/mq/pub_balancer/pub_balancer.go (renamed from weed/mq/pub_balancer/balancer.go)24
-rw-r--r--weed/mq/pub_balancer/repair.go2
-rw-r--r--weed/mq/sub_coordinator/consumer_group.go182
-rw-r--r--weed/mq/sub_coordinator/consumer_group_instance.go23
-rw-r--r--weed/mq/sub_coordinator/coordinator.go111
-rw-r--r--weed/mq/sub_coordinator/inflight_message_tracker.go160
-rw-r--r--weed/mq/sub_coordinator/inflight_message_tracker_test.go133
-rw-r--r--weed/mq/sub_coordinator/market.go367
-rw-r--r--weed/mq/sub_coordinator/market_test.go103
-rw-r--r--weed/mq/sub_coordinator/partition_consumer_mapping.go45
-rw-r--r--weed/mq/sub_coordinator/partition_consumer_mapping_test.go98
-rw-r--r--weed/mq/sub_coordinator/partition_list.go3
-rw-r--r--weed/mq/sub_coordinator/sub_coordinator.go107
-rw-r--r--weed/mq/topic/local_manager.go1
-rw-r--r--weed/mq/topic/local_partition.go48
-rw-r--r--weed/mq/topic/partition.go10
-rw-r--r--weed/pb/mq.proto61
-rw-r--r--weed/pb/mq_pb/mq.pb.go1790
-rw-r--r--weed/pb/mq_pb/mq_grpc.pb.go106
45 files changed, 3157 insertions, 1218 deletions
diff --git a/weed/filer_client/filer_client_accessor.go b/weed/filer_client/filer_client_accessor.go
new file mode 100644
index 000000000..be70f2b82
--- /dev/null
+++ b/weed/filer_client/filer_client_accessor.go
@@ -0,0 +1,65 @@
+package filer_client
+
+import (
+ "bytes"
+ "fmt"
+ "github.com/seaweedfs/seaweedfs/weed/filer"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "google.golang.org/grpc"
+ jsonpb "google.golang.org/protobuf/encoding/protojson"
+)
+
+type FilerClientAccessor struct {
+ GetFiler func() pb.ServerAddress
+ GetGrpcDialOption func() grpc.DialOption
+}
+
+func (fca *FilerClientAccessor) WithFilerClient(streamingMode bool, fn func(filer_pb.SeaweedFilerClient) error) error {
+ return pb.WithFilerClient(streamingMode, 0, fca.GetFiler(), fca.GetGrpcDialOption(), fn)
+}
+
+func (fca *FilerClientAccessor) SaveTopicConfToFiler(t *mq_pb.Topic, conf *mq_pb.ConfigureTopicResponse) error {
+
+ glog.V(0).Infof("save conf for topic %v to filer", t)
+
+ // save the topic configuration on filer
+ topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
+ if err := fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ var buf bytes.Buffer
+ filer.ProtoToText(&buf, conf)
+ return filer.SaveInsideFiler(client, topicDir, "topic.conf", buf.Bytes())
+ }); err != nil {
+ return fmt.Errorf("save topic to %s: %v", topicDir, err)
+ }
+ return nil
+}
+
+func (fca *FilerClientAccessor) ReadTopicConfFromFiler(t topic.Topic) (conf *mq_pb.ConfigureTopicResponse, err error) {
+
+ glog.V(0).Infof("load conf for topic %v from filer", t)
+
+ topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
+ if err = fca.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ data, err := filer.ReadInsideFiler(client, topicDir, "topic.conf")
+ if err == filer_pb.ErrNotFound {
+ return err
+ }
+ if err != nil {
+ return fmt.Errorf("read topic.conf of %v: %v", t, err)
+ }
+ // parse into filer conf object
+ conf = &mq_pb.ConfigureTopicResponse{}
+ if err = jsonpb.Unmarshal(data, conf); err != nil {
+ return fmt.Errorf("unmarshal topic %v conf: %v", t, err)
+ }
+ return nil
+ }); err != nil {
+ return nil, err
+ }
+
+ return conf, nil
+}
diff --git a/weed/mq/broker/broker_connect.go b/weed/mq/broker/broker_connect.go
index 3d58d89ea..386d86570 100644
--- a/weed/mq/broker/broker_connect.go
+++ b/weed/mq/broker/broker_connect.go
@@ -55,9 +55,9 @@ func (b *MessageQueueBroker) BrokerConnectToBalancer(brokerBalancer string, stop
})
if err != nil {
if err == io.EOF {
- return err
+ // return err
}
- return fmt.Errorf("send stats message: %v", err)
+ return fmt.Errorf("send stats message to balancer %s: %v", brokerBalancer, err)
}
// glog.V(3).Infof("sent stats: %+v", stats)
diff --git a/weed/mq/broker/broker_grpc_assign.go b/weed/mq/broker/broker_grpc_assign.go
index ee69db30d..48ec0d5bd 100644
--- a/weed/mq/broker/broker_grpc_assign.go
+++ b/weed/mq/broker/broker_grpc_assign.go
@@ -36,7 +36,7 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
// if is leader, notify the followers to drain existing topic partition subscriptions
if request.IsLeader {
for _, brokerPartition := range request.BrokerPartitionAssignments {
- for _, follower := range brokerPartition.FollowerBrokers {
+ if follower := brokerPartition.FollowerBroker; follower != "" {
err := pb.WithBrokerGrpcClient(false, follower, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.AssignTopicPartitions(context.Background(), request)
return err
@@ -79,11 +79,11 @@ func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context,
return fmt.Errorf("create topic %s %v on %s: %v", t, bpa.LeaderBroker, bpa.Partition, doCreateErr)
}
}
- brokerStats, found := b.Balancer.Brokers.Get(bpa.LeaderBroker)
+ brokerStats, found := b.PubBalancer.Brokers.Get(bpa.LeaderBroker)
if !found {
brokerStats = pub_balancer.NewBrokerStats()
- if !b.Balancer.Brokers.SetIfAbsent(bpa.LeaderBroker, brokerStats) {
- brokerStats, _ = b.Balancer.Brokers.Get(bpa.LeaderBroker)
+ if !b.PubBalancer.Brokers.SetIfAbsent(bpa.LeaderBroker, brokerStats) {
+ brokerStats, _ = b.PubBalancer.Brokers.Get(bpa.LeaderBroker)
}
}
brokerStats.RegisterAssignment(t, bpa.Partition, isAdd)
diff --git a/weed/mq/broker/broker_grpc_balance.go b/weed/mq/broker/broker_grpc_balance.go
index 412407211..54634c9d1 100644
--- a/weed/mq/broker/broker_grpc_balance.go
+++ b/weed/mq/broker/broker_grpc_balance.go
@@ -20,8 +20,8 @@ func (b *MessageQueueBroker) BalanceTopics(ctx context.Context, request *mq_pb.B
ret := &mq_pb.BalanceTopicsResponse{}
- actions := b.Balancer.BalancePublishers()
- err = b.Balancer.ExecuteBalanceAction(actions, b.grpcDialOption)
+ actions := b.PubBalancer.BalancePublishers()
+ err = b.PubBalancer.ExecuteBalanceAction(actions, b.grpcDialOption)
return ret, err
}
diff --git a/weed/mq/broker/broker_grpc_configure.go b/weed/mq/broker/broker_grpc_configure.go
index afb01a886..7222c8359 100644
--- a/weed/mq/broker/broker_grpc_configure.go
+++ b/weed/mq/broker/broker_grpc_configure.go
@@ -37,7 +37,7 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
t := topic.FromPbTopic(request.Topic)
var readErr, assignErr error
- resp, readErr = b.readTopicConfFromFiler(t)
+ resp, readErr = b.fca.ReadTopicConfFromFiler(t)
if readErr != nil {
glog.V(0).Infof("read topic %s conf: %v", request.Topic, readErr)
}
@@ -60,18 +60,18 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
}
}
resp = &mq_pb.ConfigureTopicResponse{}
- if b.Balancer.Brokers.IsEmpty() {
+ if b.PubBalancer.Brokers.IsEmpty() {
return nil, status.Errorf(codes.Unavailable, pub_balancer.ErrNoBroker.Error())
}
- resp.BrokerPartitionAssignments = pub_balancer.AllocateTopicPartitions(b.Balancer.Brokers, request.PartitionCount)
+ resp.BrokerPartitionAssignments = pub_balancer.AllocateTopicPartitions(b.PubBalancer.Brokers, request.PartitionCount)
resp.RecordType = request.RecordType
// save the topic configuration on filer
- if err := b.saveTopicConfToFiler(request.Topic, resp); err != nil {
+ if err := b.fca.SaveTopicConfToFiler(request.Topic, resp); err != nil {
return nil, fmt.Errorf("configure topic: %v", err)
}
- b.Balancer.OnPartitionChange(request.Topic, resp.BrokerPartitionAssignments)
+ b.PubBalancer.OnPartitionChange(request.Topic, resp.BrokerPartitionAssignments)
glog.V(0).Infof("ConfigureTopic: topic %s partition assignments: %v", request.Topic, resp.BrokerPartitionAssignments)
diff --git a/weed/mq/broker/broker_grpc_lookup.go b/weed/mq/broker/broker_grpc_lookup.go
index 14c1f37da..db62fd88a 100644
--- a/weed/mq/broker/broker_grpc_lookup.go
+++ b/weed/mq/broker/broker_grpc_lookup.go
@@ -26,7 +26,7 @@ func (b *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq
ret := &mq_pb.LookupTopicBrokersResponse{}
conf := &mq_pb.ConfigureTopicResponse{}
ret.Topic = request.Topic
- if conf, err = b.readTopicConfFromFiler(t); err != nil {
+ if conf, err = b.fca.ReadTopicConfFromFiler(t); err != nil {
glog.V(0).Infof("lookup topic %s conf: %v", request.Topic, err)
} else {
err = b.ensureTopicActiveAssignments(t, conf)
@@ -50,7 +50,7 @@ func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.List
ret := &mq_pb.ListTopicsResponse{}
knownTopics := make(map[string]struct{})
- for brokerStatsItem := range b.Balancer.Brokers.IterBuffered() {
+ for brokerStatsItem := range b.PubBalancer.Brokers.IterBuffered() {
_, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
for topicPartitionStatsItem := range brokerStats.TopicPartitionStats.IterBuffered() {
topicPartitionStat := topicPartitionStatsItem.Val
diff --git a/weed/mq/broker/broker_grpc_pub_balancer.go b/weed/mq/broker/broker_grpc_pub_balancer.go
index 2cf90b4bc..5978d2173 100644
--- a/weed/mq/broker/broker_grpc_pub_balancer.go
+++ b/weed/mq/broker/broker_grpc_pub_balancer.go
@@ -1,6 +1,7 @@
package broker
import (
+ "fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
@@ -14,32 +15,32 @@ func (b *MessageQueueBroker) PublisherToPubBalancer(stream mq_pb.SeaweedMessagin
}
req, err := stream.Recv()
if err != nil {
- return err
+ return fmt.Errorf("receive init message: %v", err)
}
// process init message
initMessage := req.GetInit()
var brokerStats *pub_balancer.BrokerStats
if initMessage != nil {
- brokerStats = b.Balancer.AddBroker(initMessage.Broker)
+ brokerStats = b.PubBalancer.AddBroker(initMessage.Broker)
} else {
return status.Errorf(codes.InvalidArgument, "balancer init message is empty")
}
defer func() {
- b.Balancer.RemoveBroker(initMessage.Broker, brokerStats)
+ b.PubBalancer.RemoveBroker(initMessage.Broker, brokerStats)
}()
// process stats message
for {
req, err := stream.Recv()
if err != nil {
- return err
+ return fmt.Errorf("receive stats message from %s: %v", initMessage.Broker, err)
}
if !b.isLockOwner() {
return status.Errorf(codes.Unavailable, "not current broker balancer")
}
if receivedStats := req.GetStats(); receivedStats != nil {
- b.Balancer.OnBrokerStatsUpdated(initMessage.Broker, brokerStats, receivedStats)
+ b.PubBalancer.OnBrokerStatsUpdated(initMessage.Broker, brokerStats, receivedStats)
// glog.V(4).Infof("received from %v: %+v", initMessage.Broker, receivedStats)
}
}
diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go
index 02488b2b0..286812a9b 100644
--- a/weed/mq/broker/broker_grpc_sub.go
+++ b/weed/mq/broker/broker_grpc_sub.go
@@ -4,24 +4,30 @@ import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
+ "io"
"time"
)
-func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest, stream mq_pb.SeaweedMessaging_SubscribeMessageServer) (err error) {
+func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_SubscribeMessageServer) error {
- ctx := stream.Context()
- clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
-
- initMessage := req.GetInit()
- if initMessage == nil {
+ req, err := stream.Recv()
+ if err != nil {
+ return err
+ }
+ if req.GetInit() == nil {
glog.Errorf("missing init message")
return fmt.Errorf("missing init message")
}
+ ctx := stream.Context()
+ clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
+
t := topic.FromPbTopic(req.GetInit().Topic)
partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
@@ -47,11 +53,98 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
}
}()
- var startPosition log_buffer.MessagePosition
- if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil {
- startPosition = getRequestPosition(req.GetInit().GetPartitionOffset())
+ startPosition := b.getRequestPosition(req.GetInit())
+ imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().Concurrency))
+
+ // connect to the follower
+ var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
+ glog.V(0).Infof("follower broker: %v", req.GetInit().FollowerBroker)
+ if req.GetInit().FollowerBroker != "" {
+ follower := req.GetInit().FollowerBroker
+ if followerGrpcConnection, err := pb.GrpcDial(ctx, follower, true, b.grpcDialOption); err != nil {
+ return fmt.Errorf("fail to dial %s: %v", follower, err)
+ } else {
+ defer func() {
+ println("closing SubscribeFollowMe connection", follower)
+ subscribeFollowMeStream.CloseSend()
+ // followerGrpcConnection.Close()
+ }()
+ followerClient := mq_pb.NewSeaweedMessagingClient(followerGrpcConnection)
+ if subscribeFollowMeStream, err = followerClient.SubscribeFollowMe(ctx); err != nil {
+ return fmt.Errorf("fail to subscribe to %s: %v", follower, err)
+ } else {
+ if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
+ Message: &mq_pb.SubscribeFollowMeRequest_Init{
+ Init: &mq_pb.SubscribeFollowMeRequest_InitMessage{
+ Topic: req.GetInit().Topic,
+ Partition: req.GetInit().GetPartitionOffset().Partition,
+ ConsumerGroup: req.GetInit().ConsumerGroup,
+ },
+ },
+ }); err != nil {
+ return fmt.Errorf("fail to send init to %s: %v", follower, err)
+ }
+ }
+ }
+ glog.V(0).Infof("follower %s connected", follower)
}
+ go func() {
+ var lastOffset int64
+ for {
+ ack, err := stream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ // the client has called CloseSend(). This is to ack the close.
+ stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Ctrl{
+ Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
+ IsEndOfStream: true,
+ },
+ }})
+ break
+ }
+ glog.V(0).Infof("topic %v partition %v subscriber %s lastOffset %d error: %v", t, partition, clientName, lastOffset, err)
+ break
+ }
+ if ack.GetAck().Key == nil {
+ // skip ack for control messages
+ continue
+ }
+ imt.AcknowledgeMessage(ack.GetAck().Key, ack.GetAck().Sequence)
+ currentLastOffset := imt.GetOldestAckedTimestamp()
+ // fmt.Printf("%+v recv (%s,%d), oldest %d\n", partition, string(ack.GetAck().Key), ack.GetAck().Sequence, currentLastOffset)
+ if subscribeFollowMeStream != nil && currentLastOffset > lastOffset {
+ if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
+ Message: &mq_pb.SubscribeFollowMeRequest_Ack{
+ Ack: &mq_pb.SubscribeFollowMeRequest_AckMessage{
+ TsNs: currentLastOffset,
+ },
+ },
+ }); err != nil {
+ glog.Errorf("Error sending ack to follower: %v", err)
+ break
+ }
+ lastOffset = currentLastOffset
+ // fmt.Printf("%+v forwarding ack %d\n", partition, lastOffset)
+ }
+ }
+ if lastOffset > 0 {
+ glog.V(0).Infof("saveConsumerGroupOffset %v %v %v %v", t, partition, req.GetInit().ConsumerGroup, lastOffset)
+ if err := b.saveConsumerGroupOffset(t, partition, req.GetInit().ConsumerGroup, lastOffset); err != nil {
+ glog.Errorf("saveConsumerGroupOffset partition %v lastOffset %d: %v", partition, lastOffset, err)
+ }
+ }
+ if subscribeFollowMeStream != nil {
+ if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
+ Message: &mq_pb.SubscribeFollowMeRequest_Close{
+ Close: &mq_pb.SubscribeFollowMeRequest_CloseMessage{},
+ },
+ }); err != nil {
+ glog.Errorf("Error sending close to follower: %v", err)
+ }
+ }
+ }()
+
return localTopicPartition.Subscribe(clientName, startPosition, func() bool {
if !isConnected {
return false
@@ -81,6 +174,13 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
// reset the sleep interval count
sleepIntervalCount = 0
+ for imt.IsInflight(logEntry.Key) {
+ time.Sleep(137 * time.Millisecond)
+ }
+ if logEntry.Key != nil {
+ imt.EnflightMessage(logEntry.Key, logEntry.TsNs)
+ }
+
if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{
Data: &mq_pb.DataMessage{
Key: logEntry.Key,
@@ -97,10 +197,21 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
})
}
-func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer.MessagePosition) {
+func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (startPosition log_buffer.MessagePosition) {
+ if initMessage == nil {
+ return
+ }
+ offset := initMessage.GetPartitionOffset()
if offset.StartTsNs != 0 {
startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
+ return
+ }
+ if storedOffset, err := b.readConsumerGroupOffset(initMessage); err == nil {
+ glog.V(0).Infof("resume from saved offset %v %v %v: %v", initMessage.Topic, initMessage.PartitionOffset.Partition, initMessage.ConsumerGroup, storedOffset)
+ startPosition = log_buffer.NewMessagePosition(storedOffset, -2)
+ return
}
+
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
startPosition = log_buffer.NewMessagePosition(1, -3)
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
diff --git a/weed/mq/broker/broker_grpc_sub_coordinator.go b/weed/mq/broker/broker_grpc_sub_coordinator.go
index 3fd97f1c2..90253b892 100644
--- a/weed/mq/broker/broker_grpc_sub_coordinator.go
+++ b/weed/mq/broker/broker_grpc_sub_coordinator.go
@@ -4,7 +4,6 @@ import (
"context"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
- "github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@@ -21,41 +20,42 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
}
var cgi *sub_coordinator.ConsumerGroupInstance
+ var cg *sub_coordinator.ConsumerGroup
// process init message
initMessage := req.GetInit()
if initMessage != nil {
- cgi = b.Coordinator.AddSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
+ cg, cgi, err = b.SubCoordinator.AddSubscriber(initMessage)
+ if err != nil {
+ return status.Errorf(codes.InvalidArgument, err.Error())
+ }
glog.V(0).Infof("subscriber %s/%s/%s connected", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
} else {
return status.Errorf(codes.InvalidArgument, "subscriber init message is empty")
}
defer func() {
- b.Coordinator.RemoveSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
+ b.SubCoordinator.RemoveSubscriber(initMessage)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}()
ctx := stream.Context()
go func() {
- // try to load the partition assignment from filer
- if conf, err := b.readTopicConfFromFiler(topic.FromPbTopic(initMessage.Topic)); err == nil {
- // send partition assignment to subscriber
- cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
- Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
- Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
- PartitionAssignments: conf.BrokerPartitionAssignments,
- },
- },
- }
- }
-
// process ack messages
for {
- _, err := stream.Recv()
+ req, err := stream.Recv()
if err != nil {
glog.V(0).Infof("subscriber %s/%s/%s receive: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}
+ if ackUnAssignment := req.GetAckUnAssignment(); ackUnAssignment != nil {
+ glog.V(0).Infof("subscriber %s/%s/%s ack close of %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, ackUnAssignment)
+ cg.AckUnAssignment(cgi, ackUnAssignment)
+ }
+ if ackAssignment := req.GetAckAssignment(); ackAssignment != nil {
+ glog.V(0).Infof("subscriber %s/%s/%s ack assignment %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, ackAssignment)
+ cg.AckAssignment(cgi, ackAssignment)
+ }
+
select {
case <-ctx.Done():
err := ctx.Err()
diff --git a/weed/mq/broker/broker_grpc_sub_follow.go b/weed/mq/broker/broker_grpc_sub_follow.go
new file mode 100644
index 000000000..f7f4ac7e9
--- /dev/null
+++ b/weed/mq/broker/broker_grpc_sub_follow.go
@@ -0,0 +1,101 @@
+package broker
+
+import (
+ "fmt"
+ "github.com/seaweedfs/seaweedfs/weed/filer"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "github.com/seaweedfs/seaweedfs/weed/util"
+ "io"
+ "time"
+)
+
+func (b *MessageQueueBroker) SubscribeFollowMe(stream mq_pb.SeaweedMessaging_SubscribeFollowMeServer) (err error) {
+ var req *mq_pb.SubscribeFollowMeRequest
+ req, err = stream.Recv()
+ if err != nil {
+ return err
+ }
+ initMessage := req.GetInit()
+ if initMessage == nil {
+ return fmt.Errorf("missing init message")
+ }
+
+ // create an in-memory offset
+ var lastOffset int64
+
+ // follow each published messages
+ for {
+ // receive a message
+ req, err = stream.Recv()
+ if err != nil {
+ if err == io.EOF {
+ err = nil
+ break
+ }
+ glog.V(0).Infof("topic %v partition %v subscribe stream error: %v", initMessage.Topic, initMessage.Partition, err)
+ break
+ }
+
+ // Process the received message
+ if ackMessage := req.GetAck(); ackMessage != nil {
+ lastOffset = ackMessage.TsNs
+ // println("sub follower got offset", lastOffset)
+ } else if closeMessage := req.GetClose(); closeMessage != nil {
+ glog.V(0).Infof("topic %v partition %v subscribe stream closed: %v", initMessage.Topic, initMessage.Partition, closeMessage)
+ return nil
+ } else {
+ glog.Errorf("unknown message: %v", req)
+ }
+ }
+
+ t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
+
+ if lastOffset > 0 {
+ err = b.saveConsumerGroupOffset(t, p, initMessage.ConsumerGroup, lastOffset)
+ }
+
+ glog.V(0).Infof("shut down follower for %v offset %d", initMessage, lastOffset)
+
+ return err
+}
+
+func (b *MessageQueueBroker) readConsumerGroupOffset(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (offset int64, err error) {
+ t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.PartitionOffset.Partition)
+
+ topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
+ partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
+ partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
+ offsetFileName := fmt.Sprintf("%s.offset", initMessage.ConsumerGroup)
+
+ err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ data, err := filer.ReadInsideFiler(client, partitionDir, offsetFileName)
+ if err != nil {
+ return err
+ }
+ if len(data) != 8 {
+ return fmt.Errorf("no offset found")
+ }
+ offset = int64(util.BytesToUint64(data))
+ return nil
+ })
+ return offset, err
+}
+
+func (b *MessageQueueBroker) saveConsumerGroupOffset(t topic.Topic, p topic.Partition, consumerGroup string, offset int64) error {
+
+ topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
+ partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
+ partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
+ offsetFileName := fmt.Sprintf("%s.offset", consumerGroup)
+
+ offsetBytes := make([]byte, 8)
+ util.Uint64toBytes(offsetBytes, uint64(offset))
+
+ return b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ glog.V(0).Infof("saving topic %s partition %v consumer group %s offset %d", t, p, consumerGroup, offset)
+ return filer.SaveInsideFiler(client, partitionDir, offsetFileName, offsetBytes)
+ })
+}
diff --git a/weed/mq/broker/broker_server.go b/weed/mq/broker/broker_server.go
index 9e62fe996..d80fa91a4 100644
--- a/weed/mq/broker/broker_server.go
+++ b/weed/mq/broker/broker_server.go
@@ -2,6 +2,7 @@ package broker
import (
"context"
+ "github.com/seaweedfs/seaweedfs/weed/filer_client"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
@@ -44,16 +45,17 @@ type MessageQueueBroker struct {
filers map[pb.ServerAddress]struct{}
currentFiler pb.ServerAddress
localTopicManager *topic.LocalTopicManager
- Balancer *pub_balancer.Balancer
+ PubBalancer *pub_balancer.PubBalancer
lockAsBalancer *cluster.LiveLock
- Coordinator *sub_coordinator.Coordinator
+ SubCoordinator *sub_coordinator.SubCoordinator
accessLock sync.Mutex
+ fca *filer_client.FilerClientAccessor
}
func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.DialOption) (mqBroker *MessageQueueBroker, err error) {
- pub_broker_balancer := pub_balancer.NewBalancer()
- coordinator := sub_coordinator.NewCoordinator(pub_broker_balancer)
+ pubBalancer := pub_balancer.NewPubBalancer()
+ subCoordinator := sub_coordinator.NewSubCoordinator()
mqBroker = &MessageQueueBroker{
option: option,
@@ -61,13 +63,18 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial
MasterClient: wdclient.NewMasterClient(grpcDialOption, option.FilerGroup, cluster.BrokerType, option.BrokerAddress(), option.DataCenter, option.Rack, *pb.NewServiceDiscoveryFromMap(option.Masters)),
filers: make(map[pb.ServerAddress]struct{}),
localTopicManager: topic.NewLocalTopicManager(),
- Balancer: pub_broker_balancer,
- Coordinator: coordinator,
+ PubBalancer: pubBalancer,
+ SubCoordinator: subCoordinator,
}
+ fca := &filer_client.FilerClientAccessor{
+ GetFiler: mqBroker.GetFiler,
+ GetGrpcDialOption: mqBroker.GetGrpcDialOption,
+ }
+ mqBroker.fca = fca
+ subCoordinator.FilerClientAccessor = fca
+
mqBroker.MasterClient.SetOnPeerUpdateFn(mqBroker.OnBrokerUpdate)
- pub_broker_balancer.OnPartitionChange = mqBroker.Coordinator.OnPartitionChange
- pub_broker_balancer.OnAddBroker = mqBroker.Coordinator.OnSubAddBroker
- pub_broker_balancer.OnRemoveBroker = mqBroker.Coordinator.OnSubRemoveBroker
+ pubBalancer.OnPartitionChange = mqBroker.SubCoordinator.OnPartitionChange
go mqBroker.MasterClient.KeepConnectedToMaster(context.Background())
@@ -119,6 +126,10 @@ func (b *MessageQueueBroker) OnBrokerUpdate(update *master_pb.ClusterNodeUpdate,
}
+func (b *MessageQueueBroker) GetGrpcDialOption() grpc.DialOption {
+ return b.grpcDialOption
+}
+
func (b *MessageQueueBroker) GetFiler() pb.ServerAddress {
return b.currentFiler
}
diff --git a/weed/mq/broker/broker_topic_conf_read_write.go b/weed/mq/broker/broker_topic_conf_read_write.go
index cddd6cf1c..ea5cb71b9 100644
--- a/weed/mq/broker/broker_topic_conf_read_write.go
+++ b/weed/mq/broker/broker_topic_conf_read_write.go
@@ -1,64 +1,16 @@
package broker
import (
- "bytes"
"fmt"
- "github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
- "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
- jsonpb "google.golang.org/protobuf/encoding/protojson"
)
-func (b *MessageQueueBroker) saveTopicConfToFiler(t *mq_pb.Topic, conf *mq_pb.ConfigureTopicResponse) error {
-
- glog.V(0).Infof("save conf for topic %v to filer", t)
-
- // save the topic configuration on filer
- topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
- if err := b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
- var buf bytes.Buffer
- filer.ProtoToText(&buf, conf)
- return filer.SaveInsideFiler(client, topicDir, "topic.conf", buf.Bytes())
- }); err != nil {
- return fmt.Errorf("save topic to %s: %v", topicDir, err)
- }
- return nil
-}
-
-// readTopicConfFromFiler reads the topic configuration from filer
-// this should only be run in broker leader, to ensure correct active broker list.
-func (b *MessageQueueBroker) readTopicConfFromFiler(t topic.Topic) (conf *mq_pb.ConfigureTopicResponse, err error) {
-
- glog.V(0).Infof("load conf for topic %v from filer", t)
-
- topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
- if err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
- data, err := filer.ReadInsideFiler(client, topicDir, "topic.conf")
- if err == filer_pb.ErrNotFound {
- return err
- }
- if err != nil {
- return fmt.Errorf("read topic.conf of %v: %v", t, err)
- }
- // parse into filer conf object
- conf = &mq_pb.ConfigureTopicResponse{}
- if err = jsonpb.Unmarshal(data, conf); err != nil {
- return fmt.Errorf("unmarshal topic %v conf: %v", t, err)
- }
- return nil
- }); err != nil {
- return nil, err
- }
-
- return conf, nil
-}
-
func (b *MessageQueueBroker) GetOrGenerateLocalPartition(t topic.Topic, partition topic.Partition) (localTopicPartition *topic.LocalPartition, getOrGenError error) {
// get or generate a local partition
- conf, readConfErr := b.readTopicConfFromFiler(t)
+ conf, readConfErr := b.fca.ReadTopicConfFromFiler(t)
if readConfErr != nil {
glog.Errorf("topic %v not found: %v", t, readConfErr)
return nil, fmt.Errorf("topic %v not found: %v", t, readConfErr)
@@ -100,10 +52,10 @@ func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition
func (b *MessageQueueBroker) ensureTopicActiveAssignments(t topic.Topic, conf *mq_pb.ConfigureTopicResponse) (err error) {
// also fix assignee broker if invalid
- hasChanges := pub_balancer.EnsureAssignmentsToActiveBrokers(b.Balancer.Brokers, 1, conf.BrokerPartitionAssignments)
+ hasChanges := pub_balancer.EnsureAssignmentsToActiveBrokers(b.PubBalancer.Brokers, 1, conf.BrokerPartitionAssignments)
if hasChanges {
glog.V(0).Infof("topic %v partition updated assignments: %v", t, conf.BrokerPartitionAssignments)
- if err = b.saveTopicConfToFiler(t.ToPbTopic(), conf); err != nil {
+ if err = b.fca.SaveTopicConfToFiler(t.ToPbTopic(), conf); err != nil {
return err
}
}
diff --git a/weed/mq/broker/broker_topic_partition_read_write.go b/weed/mq/broker/broker_topic_partition_read_write.go
index cb55e2032..7dd78c582 100644
--- a/weed/mq/broker/broker_topic_partition_read_write.go
+++ b/weed/mq/broker/broker_topic_partition_read_write.go
@@ -49,8 +49,7 @@ func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Par
localPartition.NotifyLogFlushed(logBuffer.LastFlushTsNs)
}
- println("flushing at", logBuffer.LastFlushTsNs, "to", targetFile, "size", len(buf))
-
+ glog.V(0).Infof("flushing at %d to %s size %d", logBuffer.LastFlushTsNs, targetFile, len(buf))
}
}
@@ -104,7 +103,7 @@ func (b *MessageQueueBroker) genLogOnDiskReadFunc(t topic.Topic, partition *mq_p
eachFileFn := func(entry *filer_pb.Entry, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
if len(entry.Content) > 0 {
- glog.Warningf("this should not happen. unexpected content in %s/%s", partitionDir, entry.Name)
+ // skip .offset files
return
}
var urlStrings []string
diff --git a/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go b/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go
index adcdda04c..902e7ed1b 100644
--- a/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go
+++ b/weed/mq/client/cmd/weed_sub_kv/subscriber_kv.go
@@ -14,9 +14,11 @@ import (
)
var (
- namespace = flag.String("ns", "test", "namespace")
- t = flag.String("topic", "test", "topic")
- seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
+ namespace = flag.String("ns", "test", "namespace")
+ t = flag.String("topic", "test", "topic")
+ seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
+ maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count")
+ perPartitionConcurrency = flag.Int("perPartitionConcurrency", 1, "per partition concurrency")
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
)
@@ -25,10 +27,11 @@ func main() {
flag.Parse()
subscriberConfig := &sub_client.SubscriberConfiguration{
- ClientId: fmt.Sprintf("client-%d", *clientId),
ConsumerGroup: "test",
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
+ MaxPartitionCount: int32(*maxPartitionCount),
+ PerPartitionConcurrency: int32(*perPartitionConcurrency),
}
contentConfig := &sub_client.ContentConfiguration{
@@ -37,18 +40,14 @@ func main() {
StartTime: time.Unix(1, 1),
}
- processorConfig := sub_client.ProcessorConfiguration{
- ConcurrentPartitionLimit: 3,
- }
-
brokers := strings.Split(*seedBrokers, ",")
- subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, processorConfig)
+ subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
counter := 0
- subscriber.SetEachMessageFunc(func(key, value []byte) (bool, error) {
+ subscriber.SetEachMessageFunc(func(key, value []byte) error {
counter++
println(string(key), "=>", string(value), counter)
- return true, nil
+ return nil
})
subscriber.SetCompletionFunc(func() {
diff --git a/weed/mq/client/cmd/weed_sub_record/subscriber_record.go b/weed/mq/client/cmd/weed_sub_record/subscriber_record.go
index 53eb4f15b..674c881ba 100644
--- a/weed/mq/client/cmd/weed_sub_record/subscriber_record.go
+++ b/weed/mq/client/cmd/weed_sub_record/subscriber_record.go
@@ -16,9 +16,11 @@ import (
)
var (
- namespace = flag.String("ns", "test", "namespace")
- t = flag.String("topic", "test", "topic")
- seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
+ namespace = flag.String("ns", "test", "namespace")
+ t = flag.String("topic", "test", "topic")
+ seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
+ maxPartitionCount = flag.Int("maxPartitionCount", 3, "max partition count")
+ perPartitionConcurrency = flag.Int("perPartitionConcurrency", 1, "per partition concurrency")
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
)
@@ -51,10 +53,11 @@ func main() {
flag.Parse()
subscriberConfig := &sub_client.SubscriberConfiguration{
- ClientId: fmt.Sprintf("client-%d", *clientId),
ConsumerGroup: "test",
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
+ MaxPartitionCount: int32(*maxPartitionCount),
+ PerPartitionConcurrency: int32(*perPartitionConcurrency),
}
contentConfig := &sub_client.ContentConfiguration{
@@ -63,20 +66,17 @@ func main() {
StartTime: time.Unix(1, 1),
}
- processorConfig := sub_client.ProcessorConfiguration{
- ConcurrentPartitionLimit: 3,
- }
-
brokers := strings.Split(*seedBrokers, ",")
- subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, processorConfig)
+ subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
counter := 0
- subscriber.SetEachMessageFunc(func(key, value []byte) (bool, error) {
+ subscriber.SetEachMessageFunc(func(key, value []byte) error {
counter++
record := &schema_pb.RecordValue{}
proto.Unmarshal(value, record)
fmt.Printf("record: %v\n", record)
- return true, nil
+ time.Sleep(1300 * time.Millisecond)
+ return nil
})
subscriber.SetCompletionFunc(func() {
diff --git a/weed/mq/client/pub_client/publish.go b/weed/mq/client/pub_client/publish.go
index a25620de1..a85eec31f 100644
--- a/weed/mq/client/pub_client/publish.go
+++ b/weed/mq/client/pub_client/publish.go
@@ -51,6 +51,7 @@ func (p *TopicPublisher) FinishPublish() error {
TsNs: time.Now().UnixNano(),
Ctrl: &mq_pb.ControlMessage{
IsClose: true,
+ PublisherName: p.config.PublisherName,
},
})
}
diff --git a/weed/mq/client/pub_client/scheduler.go b/weed/mq/client/pub_client/scheduler.go
index 03377d653..df2270b2c 100644
--- a/weed/mq/client/pub_client/scheduler.go
+++ b/weed/mq/client/pub_client/scheduler.go
@@ -142,11 +142,11 @@ func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) erro
if err = publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Init{
Init: &mq_pb.PublishMessageRequest_InitMessage{
- Topic: p.config.Topic.ToPbTopic(),
- Partition: job.Partition,
- AckInterval: 128,
- FollowerBrokers: job.FollowerBrokers,
- PublisherName: p.config.PublisherName,
+ Topic: p.config.Topic.ToPbTopic(),
+ Partition: job.Partition,
+ AckInterval: 128,
+ FollowerBroker: job.FollowerBroker,
+ PublisherName: p.config.PublisherName,
},
},
}); err != nil {
diff --git a/weed/mq/client/sub_client/connect_to_sub_coordinator.go b/weed/mq/client/sub_client/connect_to_sub_coordinator.go
index 2f1330b5e..d05ddb960 100644
--- a/weed/mq/client/sub_client/connect_to_sub_coordinator.go
+++ b/weed/mq/client/sub_client/connect_to_sub_coordinator.go
@@ -2,12 +2,9 @@ package sub_client
import (
"context"
- "fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
- "io"
- "sync"
"time"
)
@@ -51,6 +48,7 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerGroupInstanceId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: sub.ContentConfig.Topic.ToPbTopic(),
+ MaxPartitionCount: sub.SubscriberConfig.MaxPartitionCount,
},
},
}); err != nil {
@@ -58,6 +56,16 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
return err
}
+ go func() {
+ for reply := range sub.brokerPartitionAssignmentAckChan {
+ glog.V(0).Infof("subscriber instance %s ack %+v", sub.SubscriberConfig.ConsumerGroupInstanceId, reply)
+ if err := stream.Send(reply); err != nil {
+ glog.V(0).Infof("subscriber %s reply: %v", sub.ContentConfig.Topic, err)
+ return
+ }
+ }
+ }()
+
// keep receiving messages from the sub coordinator
for {
resp, err := stream.Recv()
@@ -65,11 +73,8 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
glog.V(0).Infof("subscriber %s receive: %v", sub.ContentConfig.Topic, err)
return err
}
- assignment := resp.GetAssignment()
- if assignment != nil {
- glog.V(0).Infof("subscriber %s receive assignment: %v", sub.ContentConfig.Topic, assignment)
- }
- sub.onEachAssignment(assignment)
+ sub.brokerPartitionAssignmentChan <- resp
+ glog.V(0).Infof("Received assignment: %+v", resp)
}
return nil
@@ -82,100 +87,3 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
time.Sleep(waitTime)
}
}
-
-func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCoordinatorResponse_Assignment) {
- if assignment == nil {
- return
- }
- // process each partition, with a concurrency limit
- var wg sync.WaitGroup
- semaphore := make(chan struct{}, sub.ProcessorConfig.ConcurrentPartitionLimit)
-
- for _, assigned := range assignment.PartitionAssignments {
- wg.Add(1)
- semaphore <- struct{}{}
- go func(assigned *mq_pb.BrokerPartitionAssignment) {
- defer wg.Done()
- defer func() { <-semaphore }()
- glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
- err := sub.onEachPartition(assigned)
- if err != nil {
- glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
- }
- }(assigned)
- }
-
- wg.Wait()
-}
-
-func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment) error {
- // connect to the partition broker
- return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
- subscribeClient, err := client.SubscribeMessage(context.Background(), &mq_pb.SubscribeMessageRequest{
- Message: &mq_pb.SubscribeMessageRequest_Init{
- Init: &mq_pb.SubscribeMessageRequest_InitMessage{
- ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
- ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
- Topic: sub.ContentConfig.Topic.ToPbTopic(),
- PartitionOffset: &mq_pb.PartitionOffset{
- Partition: assigned.Partition,
- StartTsNs: sub.alreadyProcessedTsNs,
- StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
- },
- Filter: sub.ContentConfig.Filter,
- FollowerBrokers: assigned.FollowerBrokers,
- },
- },
- })
-
- if err != nil {
- return fmt.Errorf("create subscribe client: %v", err)
- }
-
- glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
-
- if sub.OnCompletionFunc != nil {
- defer sub.OnCompletionFunc()
- }
- defer func() {
- subscribeClient.SendMsg(&mq_pb.SubscribeMessageRequest{
- Message: &mq_pb.SubscribeMessageRequest_Ack{
- Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
- Sequence: 0,
- },
- },
- })
- subscribeClient.CloseSend()
- }()
-
- for {
- // glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
- resp, err := subscribeClient.Recv()
- if err != nil {
- return fmt.Errorf("subscribe recv: %v", err)
- }
- if resp.Message == nil {
- glog.V(0).Infof("subscriber %s/%s received nil message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
- continue
- }
- switch m := resp.Message.(type) {
- case *mq_pb.SubscribeMessageResponse_Data:
- shouldContinue, processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
- if processErr != nil {
- return fmt.Errorf("process error: %v", processErr)
- }
- sub.alreadyProcessedTsNs = m.Data.TsNs
- if !shouldContinue {
- return nil
- }
- case *mq_pb.SubscribeMessageResponse_Ctrl:
- // glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl)
- if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
- return io.EOF
- }
- }
- }
-
- return nil
- })
-}
diff --git a/weed/mq/client/sub_client/on_each_partition.go b/weed/mq/client/sub_client/on_each_partition.go
new file mode 100644
index 000000000..5dcac4eb3
--- /dev/null
+++ b/weed/mq/client/sub_client/on_each_partition.go
@@ -0,0 +1,125 @@
+package sub_client
+
+import (
+ "context"
+ "fmt"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/pb"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "github.com/seaweedfs/seaweedfs/weed/util"
+ "io"
+)
+
+func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}) error {
+ // connect to the partition broker
+ return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
+
+ subscribeClient, err := client.SubscribeMessage(context.Background())
+ if err != nil {
+ return fmt.Errorf("create subscribe client: %v", err)
+ }
+
+ perPartitionConcurrency := sub.SubscriberConfig.PerPartitionConcurrency
+ if perPartitionConcurrency <= 0 {
+ perPartitionConcurrency = 1
+ }
+
+ if err = subscribeClient.Send(&mq_pb.SubscribeMessageRequest{
+ Message: &mq_pb.SubscribeMessageRequest_Init{
+ Init: &mq_pb.SubscribeMessageRequest_InitMessage{
+ ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
+ ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
+ Topic: sub.ContentConfig.Topic.ToPbTopic(),
+ PartitionOffset: &mq_pb.PartitionOffset{
+ Partition: assigned.Partition,
+ StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
+ },
+ Filter: sub.ContentConfig.Filter,
+ FollowerBroker: assigned.FollowerBroker,
+ Concurrency: perPartitionConcurrency,
+ },
+ },
+ }); err != nil {
+ glog.V(0).Infof("subscriber %s connected to partition %+v at %v: %v", sub.ContentConfig.Topic, assigned.Partition, assigned.LeaderBroker, err)
+ }
+
+ glog.V(0).Infof("subscriber %s connected to partition %+v at %v", sub.ContentConfig.Topic, assigned.Partition, assigned.LeaderBroker)
+
+ if sub.OnCompletionFunc != nil {
+ defer sub.OnCompletionFunc()
+ }
+
+ type KeyedOffset struct {
+ Key []byte
+ Offset int64
+ }
+
+ partitionOffsetChan := make(chan KeyedOffset, 1024)
+ defer func() {
+ close(partitionOffsetChan)
+ }()
+ executors := util.NewLimitedConcurrentExecutor(int(perPartitionConcurrency))
+
+ go func() {
+ for {
+ select {
+ case <-stopCh:
+ subscribeClient.CloseSend()
+ return
+ case ack, ok := <-partitionOffsetChan:
+ if !ok {
+ subscribeClient.CloseSend()
+ return
+ }
+ subscribeClient.SendMsg(&mq_pb.SubscribeMessageRequest{
+ Message: &mq_pb.SubscribeMessageRequest_Ack{
+ Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
+ Key: ack.Key,
+ Sequence: ack.Offset,
+ },
+ },
+ })
+ }
+ }
+ }()
+
+ var lastErr error
+
+ for lastErr == nil {
+ // glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
+ resp, err := subscribeClient.Recv()
+ if err != nil {
+ return fmt.Errorf("subscribe recv: %v", err)
+ }
+ if resp.Message == nil {
+ glog.V(0).Infof("subscriber %s/%s received nil message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
+ continue
+ }
+ switch m := resp.Message.(type) {
+ case *mq_pb.SubscribeMessageResponse_Data:
+ if m.Data.Ctrl != nil {
+ glog.V(2).Infof("subscriber %s received control from producer:%s isClose:%v", sub.SubscriberConfig.ConsumerGroup, m.Data.Ctrl.IsClose)
+ continue
+ }
+ executors.Execute(func() {
+ processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
+ if processErr == nil {
+ partitionOffsetChan <- KeyedOffset{
+ Key: m.Data.Key,
+ Offset: m.Data.TsNs,
+ }
+ } else {
+ lastErr = processErr
+ }
+ })
+ case *mq_pb.SubscribeMessageResponse_Ctrl:
+ // glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl)
+ if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
+ return io.EOF
+ }
+ }
+ }
+
+ return lastErr
+ })
+}
diff --git a/weed/mq/client/sub_client/subscribe.go b/weed/mq/client/sub_client/subscribe.go
index df62ea674..5669bb348 100644
--- a/weed/mq/client/sub_client/subscribe.go
+++ b/weed/mq/client/sub_client/subscribe.go
@@ -1,11 +1,121 @@
package sub_client
+import (
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "sync"
+ "time"
+)
+
+type ProcessorState struct {
+ stopCh chan struct{}
+}
+
// Subscribe subscribes to a topic's specified partitions.
// If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker.
func (sub *TopicSubscriber) Subscribe() error {
+
+ go sub.startProcessors()
+
// loop forever
sub.doKeepConnectedToSubCoordinator()
return nil
}
+
+func (sub *TopicSubscriber) startProcessors() {
+ // listen to the messages from the sub coordinator
+ // start one processor per partition
+ var wg sync.WaitGroup
+ semaphore := make(chan struct{}, sub.SubscriberConfig.MaxPartitionCount)
+
+ for message := range sub.brokerPartitionAssignmentChan {
+ if assigned := message.GetAssignment(); assigned != nil {
+ wg.Add(1)
+ semaphore <- struct{}{}
+
+ topicPartition := topic.FromPbPartition(assigned.PartitionAssignment.Partition)
+
+ // wait until no covering partition is still in progress
+ sub.waitUntilNoOverlappingPartitionInFlight(topicPartition)
+
+ // start a processors
+ stopChan := make(chan struct{})
+ sub.activeProcessorsLock.Lock()
+ sub.activeProcessors[topicPartition] = &ProcessorState{
+ stopCh: stopChan,
+ }
+ sub.activeProcessorsLock.Unlock()
+
+ go func(assigned *mq_pb.BrokerPartitionAssignment, topicPartition topic.Partition) {
+ defer func() {
+ sub.activeProcessorsLock.Lock()
+ delete(sub.activeProcessors, topicPartition)
+ sub.activeProcessorsLock.Unlock()
+
+ <-semaphore
+ wg.Done()
+ }()
+ glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
+ sub.brokerPartitionAssignmentAckChan <- &mq_pb.SubscriberToSubCoordinatorRequest{
+ Message: &mq_pb.SubscriberToSubCoordinatorRequest_AckAssignment{
+ AckAssignment: &mq_pb.SubscriberToSubCoordinatorRequest_AckAssignmentMessage{
+ Partition: assigned.Partition,
+ },
+ },
+ }
+ err := sub.onEachPartition(assigned, stopChan)
+ if err != nil {
+ glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
+ } else {
+ glog.V(0).Infof("subscriber %s/%s partition %+v at %v completed", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
+ }
+ sub.brokerPartitionAssignmentAckChan <- &mq_pb.SubscriberToSubCoordinatorRequest{
+ Message: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignment{
+ AckUnAssignment: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{
+ Partition: assigned.Partition,
+ },
+ },
+ }
+ }(assigned.PartitionAssignment, topicPartition)
+ }
+ if unAssignment := message.GetUnAssignment(); unAssignment != nil {
+ topicPartition := topic.FromPbPartition(unAssignment.Partition)
+ sub.activeProcessorsLock.Lock()
+ if processor, found := sub.activeProcessors[topicPartition]; found {
+ close(processor.stopCh)
+ delete(sub.activeProcessors, topicPartition)
+ }
+ sub.activeProcessorsLock.Unlock()
+ }
+ }
+
+ wg.Wait()
+
+}
+
+func (sub *TopicSubscriber) waitUntilNoOverlappingPartitionInFlight(topicPartition topic.Partition) {
+ foundOverlapping := true
+ for foundOverlapping {
+ sub.activeProcessorsLock.Lock()
+ foundOverlapping = false
+ var overlappedPartition topic.Partition
+ for partition, _ := range sub.activeProcessors {
+ if partition.Overlaps(topicPartition) {
+ if partition.Equals(topicPartition) {
+ continue
+ }
+ foundOverlapping = true
+ overlappedPartition = partition
+ break
+ }
+ }
+ sub.activeProcessorsLock.Unlock()
+ if foundOverlapping {
+ glog.V(0).Infof("subscriber %s new partition %v waiting for partition %+v to complete", sub.ContentConfig.Topic, topicPartition, overlappedPartition)
+ time.Sleep(1 * time.Second)
+ }
+ }
+}
diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go
index 982c3f13b..922593b77 100644
--- a/weed/mq/client/sub_client/subscriber.go
+++ b/weed/mq/client/sub_client/subscriber.go
@@ -4,6 +4,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc"
+ "sync"
"time"
)
@@ -11,10 +12,9 @@ type SubscriberConfiguration struct {
ClientId string
ConsumerGroup string
ConsumerGroupInstanceId string
- GroupMinimumPeers int32
- GroupMaximumPeers int32
- BootstrapServers []string
GrpcDialOption grpc.DialOption
+ MaxPartitionCount int32 // how many partitions to process concurrently
+ PerPartitionConcurrency int32 // how many messages to process concurrently per partition
}
type ContentConfiguration struct {
@@ -23,33 +23,31 @@ type ContentConfiguration struct {
StartTime time.Time
}
-type ProcessorConfiguration struct {
- ConcurrentPartitionLimit int // how many partitions to process concurrently
-}
-
-type OnEachMessageFunc func(key, value []byte) (shouldContinue bool, err error)
+type OnEachMessageFunc func(key, value []byte) (err error)
type OnCompletionFunc func()
type TopicSubscriber struct {
- SubscriberConfig *SubscriberConfiguration
- ContentConfig *ContentConfiguration
- ProcessorConfig *ProcessorConfiguration
- brokerPartitionAssignments []*mq_pb.BrokerPartitionAssignment
- OnEachMessageFunc OnEachMessageFunc
- OnCompletionFunc OnCompletionFunc
- bootstrapBrokers []string
- waitForMoreMessage bool
- alreadyProcessedTsNs int64
+ SubscriberConfig *SubscriberConfiguration
+ ContentConfig *ContentConfiguration
+ brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
+ brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
+ OnEachMessageFunc OnEachMessageFunc
+ OnCompletionFunc OnCompletionFunc
+ bootstrapBrokers []string
+ waitForMoreMessage bool
+ activeProcessors map[topic.Partition]*ProcessorState
+ activeProcessorsLock sync.Mutex
}
-func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, processor ProcessorConfiguration) *TopicSubscriber {
+func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
return &TopicSubscriber{
- SubscriberConfig: subscriber,
- ContentConfig: content,
- ProcessorConfig: &processor,
- bootstrapBrokers: bootstrapBrokers,
- waitForMoreMessage: true,
- alreadyProcessedTsNs: content.StartTime.UnixNano(),
+ SubscriberConfig: subscriber,
+ ContentConfig: content,
+ brokerPartitionAssignmentChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1024),
+ brokerPartitionAssignmentAckChan: make(chan *mq_pb.SubscriberToSubCoordinatorRequest, 1024),
+ bootstrapBrokers: bootstrapBrokers,
+ waitForMoreMessage: true,
+ activeProcessors: make(map[topic.Partition]*ProcessorState),
}
}
diff --git a/weed/mq/pub_balancer/allocate.go b/weed/mq/pub_balancer/allocate.go
index d7632f8d6..ae32188ec 100644
--- a/weed/mq/pub_balancer/allocate.go
+++ b/weed/mq/pub_balancer/allocate.go
@@ -48,20 +48,11 @@ func pickBrokers(brokers cmap.ConcurrentMap[string, *BrokerStats], count int32)
return pickedBrokers
}
-// reservoir sampling select N brokers from the active brokers, with exclusion of the excluded brokers
-func pickBrokersExcluded(brokers []string, count int, excludedLeadBroker string, excludedBrokers []string) []string {
- // convert the excluded brokers to a map
- excludedBrokerMap := make(map[string]bool)
- for _, broker := range excludedBrokers {
- excludedBrokerMap[broker] = true
- }
- if excludedLeadBroker != "" {
- excludedBrokerMap[excludedLeadBroker] = true
- }
-
+// reservoir sampling select N brokers from the active brokers, with exclusion of the excluded broker
+func pickBrokersExcluded(brokers []string, count int, excludedLeadBroker string, excludedBroker string) []string {
pickedBrokers := make([]string, 0, count)
for i, broker := range brokers {
- if _, found := excludedBrokerMap[broker]; found {
+ if broker == excludedBroker {
continue
}
if len(pickedBrokers) < count {
@@ -102,21 +93,15 @@ func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *
assignment.LeaderBroker = ""
count++
}
- for i := 0; i < followerCount; i++ {
- if i >= len(assignment.FollowerBrokers) {
- count++
- continue
- }
- if assignment.FollowerBrokers[i] == "" {
- count++
- } else if _, found := activeBrokers.Get(assignment.FollowerBrokers[i]); !found {
- assignment.FollowerBrokers[i] = ""
- count++
- }
+ if assignment.FollowerBroker == "" {
+ count++
+ } else if _, found := activeBrokers.Get(assignment.FollowerBroker); !found {
+ assignment.FollowerBroker = ""
+ count++
}
if count > 0 {
- pickedBrokers := pickBrokersExcluded(candidates, count, assignment.LeaderBroker, assignment.FollowerBrokers)
+ pickedBrokers := pickBrokersExcluded(candidates, count, assignment.LeaderBroker, assignment.FollowerBroker)
i := 0
if assignment.LeaderBroker == "" {
if i < len(pickedBrokers) {
@@ -125,34 +110,13 @@ func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *
hasChanges = true
}
}
-
- hasEmptyFollowers := false
- j := 0
- for ; j < len(assignment.FollowerBrokers); j++ {
- if assignment.FollowerBrokers[j] == "" {
+ if assignment.FollowerBroker == "" {
+ if i < len(pickedBrokers) {
+ assignment.FollowerBroker = pickedBrokers[i]
+ i++
hasChanges = true
- if i < len(pickedBrokers) {
- assignment.FollowerBrokers[j] = pickedBrokers[i]
- i++
- } else {
- hasEmptyFollowers = true
- }
}
}
- if hasEmptyFollowers {
- var followerBrokers []string
- for _, follower := range assignment.FollowerBrokers {
- if follower != "" {
- followerBrokers = append(followerBrokers, follower)
- }
- }
- assignment.FollowerBrokers = followerBrokers
- }
-
- if i < len(pickedBrokers) {
- assignment.FollowerBrokers = append(assignment.FollowerBrokers, pickedBrokers[i:]...)
- hasChanges = true
- }
}
}
diff --git a/weed/mq/pub_balancer/balance.go b/weed/mq/pub_balancer/balance.go
index 87fc5739b..b4f1e20cd 100644
--- a/weed/mq/pub_balancer/balance.go
+++ b/weed/mq/pub_balancer/balance.go
@@ -54,12 +54,12 @@ type BalanceActionCreate struct {
// BalancePublishers check the stats of all brokers,
// and balance the publishers to the brokers.
-func (balancer *Balancer) BalancePublishers() []BalanceAction {
+func (balancer *PubBalancer) BalancePublishers() []BalanceAction {
action := BalanceTopicPartitionOnBrokers(balancer.Brokers)
return []BalanceAction{action}
}
-func (balancer *Balancer) ExecuteBalanceAction(actions []BalanceAction, grpcDialOption grpc.DialOption) (err error) {
+func (balancer *PubBalancer) ExecuteBalanceAction(actions []BalanceAction, grpcDialOption grpc.DialOption) (err error) {
for _, action := range actions {
switch action.(type) {
case *BalanceActionMove:
diff --git a/weed/mq/pub_balancer/balance_action.go b/weed/mq/pub_balancer/balance_action.go
index c29ec3469..a2d888b2a 100644
--- a/weed/mq/pub_balancer/balance_action.go
+++ b/weed/mq/pub_balancer/balance_action.go
@@ -8,10 +8,10 @@ import (
"google.golang.org/grpc"
)
-// Balancer <= PublisherToPubBalancer() <= Broker <=> Publish()
-// ExecuteBalanceActionMove from Balancer => AssignTopicPartitions() => Broker => Publish()
+// PubBalancer <= PublisherToPubBalancer() <= Broker <=> Publish()
+// ExecuteBalanceActionMove from PubBalancer => AssignTopicPartitions() => Broker => Publish()
-func (balancer *Balancer) ExecuteBalanceActionMove(move *BalanceActionMove, grpcDialOption grpc.DialOption) error {
+func (balancer *PubBalancer) ExecuteBalanceActionMove(move *BalanceActionMove, grpcDialOption grpc.DialOption) error {
if _, found := balancer.Brokers.Get(move.SourceBroker); !found {
return fmt.Errorf("source broker %s not found", move.SourceBroker)
}
diff --git a/weed/mq/pub_balancer/lookup.go b/weed/mq/pub_balancer/lookup.go
index 052932c04..423b38ecb 100644
--- a/weed/mq/pub_balancer/lookup.go
+++ b/weed/mq/pub_balancer/lookup.go
@@ -9,7 +9,7 @@ var (
ErrNoBroker = errors.New("no broker")
)
-func (balancer *Balancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
+func (balancer *PubBalancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
// find existing topic partition assignments
for brokerStatsItem := range balancer.Brokers.IterBuffered() {
broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
diff --git a/weed/mq/pub_balancer/partition_list_broker.go b/weed/mq/pub_balancer/partition_list_broker.go
index 441b61898..d084cf74e 100644
--- a/weed/mq/pub_balancer/partition_list_broker.go
+++ b/weed/mq/pub_balancer/partition_list_broker.go
@@ -10,6 +10,7 @@ type PartitionSlotToBroker struct {
RangeStop int32
UnixTimeNs int64
AssignedBroker string
+ FollowerBroker string
}
type PartitionSlotToBrokerList struct {
@@ -23,16 +24,18 @@ func NewPartitionSlotToBrokerList(ringSize int32) *PartitionSlotToBrokerList {
}
}
-func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string) {
+func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broker string, follower string) {
for _, partitionSlot := range ps.PartitionSlots {
if partitionSlot.RangeStart == partition.RangeStart && partitionSlot.RangeStop == partition.RangeStop {
- if partitionSlot.AssignedBroker == broker {
- return
- }
- if partitionSlot.AssignedBroker != "" {
+ if partitionSlot.AssignedBroker != "" && partitionSlot.AssignedBroker != broker {
glog.V(0).Infof("partition %s broker change: %s => %s", partition, partitionSlot.AssignedBroker, broker)
+ partitionSlot.AssignedBroker = broker
+ }
+ if partitionSlot.FollowerBroker != "" && partitionSlot.FollowerBroker != follower {
+ glog.V(0).Infof("partition %s follower change: %s => %s", partition, partitionSlot.FollowerBroker, follower)
+ partitionSlot.FollowerBroker = follower
}
- partitionSlot.AssignedBroker = broker
+
return
}
}
@@ -41,6 +44,7 @@ func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broke
RangeStop: partition.RangeStop,
UnixTimeNs: partition.UnixTimeNs,
AssignedBroker: broker,
+ FollowerBroker: follower,
})
}
func (ps *PartitionSlotToBrokerList) RemoveBroker(broker string) {
diff --git a/weed/mq/pub_balancer/balancer.go b/weed/mq/pub_balancer/pub_balancer.go
index ad894b1d8..755cc8f30 100644
--- a/weed/mq/pub_balancer/balancer.go
+++ b/weed/mq/pub_balancer/pub_balancer.go
@@ -11,7 +11,7 @@ const (
LockBrokerBalancer = "broker_balancer"
)
-// Balancer collects stats from all brokers.
+// PubBalancer collects stats from all brokers.
//
// When publishers wants to create topics, it picks brokers to assign the topic partitions.
// When consumers wants to subscribe topics, it tells which brokers are serving the topic partitions.
@@ -28,23 +28,21 @@ const (
//
// When a consumer instance is down, the broker will notice this and inform the balancer.
// The balancer will then tell the broker to send the partition to another standby consumer instance.
-type Balancer struct {
+type PubBalancer struct {
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
// Collected from all brokers when they connect to the broker leader
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
OnPartitionChange func(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment)
- OnAddBroker func(broker string, brokerStats *BrokerStats)
- OnRemoveBroker func(broker string, brokerStats *BrokerStats)
}
-func NewBalancer() *Balancer {
- return &Balancer{
+func NewPubBalancer() *PubBalancer {
+ return &PubBalancer{
Brokers: cmap.New[*BrokerStats](),
TopicToBrokers: cmap.New[*PartitionSlotToBrokerList](),
}
}
-func (balancer *Balancer) AddBroker(broker string) (brokerStats *BrokerStats) {
+func (balancer *PubBalancer) AddBroker(broker string) (brokerStats *BrokerStats) {
var found bool
brokerStats, found = balancer.Brokers.Get(broker)
if !found {
@@ -54,11 +52,10 @@ func (balancer *Balancer) AddBroker(broker string) (brokerStats *BrokerStats) {
}
}
balancer.onPubAddBroker(broker, brokerStats)
- balancer.OnAddBroker(broker, brokerStats)
return brokerStats
}
-func (balancer *Balancer) RemoveBroker(broker string, stats *BrokerStats) {
+func (balancer *PubBalancer) RemoveBroker(broker string, stats *BrokerStats) {
balancer.Brokers.Remove(broker)
// update TopicToBrokers
@@ -75,10 +72,9 @@ func (balancer *Balancer) RemoveBroker(broker string, stats *BrokerStats) {
}
}
balancer.onPubRemoveBroker(broker, stats)
- balancer.OnRemoveBroker(broker, stats)
}
-func (balancer *Balancer) OnBrokerStatsUpdated(broker string, brokerStats *BrokerStats, receivedStats *mq_pb.BrokerStats) {
+func (balancer *PubBalancer) OnBrokerStatsUpdated(broker string, brokerStats *BrokerStats, receivedStats *mq_pb.BrokerStats) {
brokerStats.UpdateStats(receivedStats)
// update TopicToBrokers
@@ -92,14 +88,14 @@ func (balancer *Balancer) OnBrokerStatsUpdated(broker string, brokerStats *Broke
partitionSlotToBrokerList, _ = balancer.TopicToBrokers.Get(topicKey)
}
}
- partitionSlotToBrokerList.AddBroker(partition, broker)
+ partitionSlotToBrokerList.AddBroker(partition, broker, topicPartitionStats.Follower)
}
}
// OnPubAddBroker is called when a broker is added for a publisher coordinator
-func (balancer *Balancer) onPubAddBroker(broker string, brokerStats *BrokerStats) {
+func (balancer *PubBalancer) onPubAddBroker(broker string, brokerStats *BrokerStats) {
}
// OnPubRemoveBroker is called when a broker is removed for a publisher coordinator
-func (balancer *Balancer) onPubRemoveBroker(broker string, brokerStats *BrokerStats) {
+func (balancer *PubBalancer) onPubRemoveBroker(broker string, brokerStats *BrokerStats) {
}
diff --git a/weed/mq/pub_balancer/repair.go b/weed/mq/pub_balancer/repair.go
index 0f307c9eb..d16715406 100644
--- a/weed/mq/pub_balancer/repair.go
+++ b/weed/mq/pub_balancer/repair.go
@@ -8,7 +8,7 @@ import (
"sort"
)
-func (balancer *Balancer) RepairTopics() []BalanceAction {
+func (balancer *PubBalancer) RepairTopics() []BalanceAction {
action := BalanceTopicPartitionOnBrokers(balancer.Brokers)
return []BalanceAction{action}
}
diff --git a/weed/mq/sub_coordinator/consumer_group.go b/weed/mq/sub_coordinator/consumer_group.go
index d24a38d8a..80a2d5f80 100644
--- a/weed/mq/sub_coordinator/consumer_group.go
+++ b/weed/mq/sub_coordinator/consumer_group.go
@@ -1,129 +1,111 @@
package sub_coordinator
import (
+ "fmt"
cmap "github.com/orcaman/concurrent-map/v2"
+ "github.com/seaweedfs/seaweedfs/weed/filer_client"
"github.com/seaweedfs/seaweedfs/weed/glog"
- "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"time"
)
-type ConsumerGroupInstance struct {
- InstanceId string
- // the consumer group instance may not have an active partition
- Partitions []*topic.Partition
- ResponseChan chan *mq_pb.SubscriberToSubCoordinatorResponse
-}
type ConsumerGroup struct {
topic topic.Topic
// map a consumer group instance id to a consumer group instance
ConsumerGroupInstances cmap.ConcurrentMap[string, *ConsumerGroupInstance]
- mapping *PartitionConsumerMapping
+ Market *Market
reBalanceTimer *time.Timer
- pubBalancer *pub_balancer.Balancer
+ filerClientAccessor *filer_client.FilerClientAccessor
+ stopCh chan struct{}
}
-func NewConsumerGroup(t *mq_pb.Topic, pubBalancer *pub_balancer.Balancer) *ConsumerGroup {
- return &ConsumerGroup{
+func NewConsumerGroup(t *mq_pb.Topic, reblanceSeconds int32, filerClientAccessor *filer_client.FilerClientAccessor) *ConsumerGroup {
+ cg := &ConsumerGroup{
topic: topic.FromPbTopic(t),
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),
- mapping: NewPartitionConsumerMapping(pub_balancer.MaxPartitionCount),
- pubBalancer: pubBalancer,
+ filerClientAccessor: filerClientAccessor,
+ stopCh: make(chan struct{}),
}
-}
-
-func NewConsumerGroupInstance(instanceId string) *ConsumerGroupInstance {
- return &ConsumerGroupInstance{
- InstanceId: instanceId,
- ResponseChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1),
+ if conf, err := cg.filerClientAccessor.ReadTopicConfFromFiler(cg.topic); err == nil {
+ var partitions []topic.Partition
+ for _, assignment := range conf.BrokerPartitionAssignments {
+ partitions = append(partitions, topic.FromPbPartition(assignment.Partition))
+ }
+ cg.Market = NewMarket(partitions, time.Duration(reblanceSeconds)*time.Second)
+ } else {
+ glog.V(0).Infof("fail to read topic conf from filer: %v", err)
+ return nil
}
-}
-func (cg *ConsumerGroup) OnAddConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
- cg.onConsumerGroupInstanceChange("add consumer instance " + consumerGroupInstance)
-}
-func (cg *ConsumerGroup) OnRemoveConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
- cg.onConsumerGroupInstanceChange("remove consumer instance " + consumerGroupInstance)
+
+ go func() {
+ for {
+ select {
+ case adjustment := <-cg.Market.AdjustmentChan:
+ cgi, found := cg.ConsumerGroupInstances.Get(string(adjustment.consumer))
+ if !found {
+ glog.V(0).Infof("consumer group instance %s not found", adjustment.consumer)
+ continue
+ }
+ if adjustment.isAssign {
+ if conf, err := cg.filerClientAccessor.ReadTopicConfFromFiler(cg.topic); err == nil {
+ for _, assignment := range conf.BrokerPartitionAssignments {
+ if adjustment.partition.Equals(topic.FromPbPartition(assignment.Partition)) {
+ cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
+ Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
+ Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
+ PartitionAssignment: &mq_pb.BrokerPartitionAssignment{
+ Partition: adjustment.partition.ToPbPartition(),
+ LeaderBroker: assignment.LeaderBroker,
+ FollowerBroker: assignment.FollowerBroker,
+ },
+ },
+ },
+ }
+ glog.V(0).Infof("send assignment %v to %s", adjustment.partition, adjustment.consumer)
+ break
+ }
+ }
+ }
+ } else {
+ cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
+ Message: &mq_pb.SubscriberToSubCoordinatorResponse_UnAssignment_{
+ UnAssignment: &mq_pb.SubscriberToSubCoordinatorResponse_UnAssignment{
+ Partition: adjustment.partition.ToPbPartition(),
+ },
+ },
+ }
+ glog.V(0).Infof("send unassignment %v to %s", adjustment.partition, adjustment.consumer)
+ }
+ case <-cg.stopCh:
+ return
+ }
+ }
+ }()
+
+ return cg
}
-func (cg *ConsumerGroup) onConsumerGroupInstanceChange(reason string) {
- if cg.reBalanceTimer != nil {
- cg.reBalanceTimer.Stop()
- cg.reBalanceTimer = nil
- }
- cg.reBalanceTimer = time.AfterFunc(5*time.Second, func() {
- cg.RebalanceConsumberGroupInstances(nil, reason)
- cg.reBalanceTimer = nil
+func (cg *ConsumerGroup) AckAssignment(cgi *ConsumerGroupInstance, assignment *mq_pb.SubscriberToSubCoordinatorRequest_AckAssignmentMessage) {
+ fmt.Printf("ack assignment %v\n", assignment)
+ cg.Market.ConfirmAdjustment(&Adjustment{
+ consumer: cgi.InstanceId,
+ partition: topic.FromPbPartition(assignment.Partition),
+ isAssign: true,
})
}
-func (cg *ConsumerGroup) OnPartitionListChange(assignments []*mq_pb.BrokerPartitionAssignment) {
- if cg.reBalanceTimer != nil {
- cg.reBalanceTimer.Stop()
- cg.reBalanceTimer = nil
- }
- partitionSlotToBrokerList := pub_balancer.NewPartitionSlotToBrokerList(pub_balancer.MaxPartitionCount)
- for _, assignment := range assignments {
- partitionSlotToBrokerList.AddBroker(assignment.Partition, assignment.LeaderBroker)
- }
- cg.RebalanceConsumberGroupInstances(partitionSlotToBrokerList, "partition list change")
+func (cg *ConsumerGroup) AckUnAssignment(cgi *ConsumerGroupInstance, assignment *mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) {
+ fmt.Printf("ack unassignment %v\n", assignment)
+ cg.Market.ConfirmAdjustment(&Adjustment{
+ consumer: cgi.InstanceId,
+ partition: topic.FromPbPartition(assignment.Partition),
+ isAssign: false,
+ })
}
-func (cg *ConsumerGroup) RebalanceConsumberGroupInstances(knownPartitionSlotToBrokerList *pub_balancer.PartitionSlotToBrokerList, reason string) {
- glog.V(0).Infof("rebalance consumer group %s due to %s", cg.topic.String(), reason)
-
- // collect current topic partitions
- partitionSlotToBrokerList := knownPartitionSlotToBrokerList
- if partitionSlotToBrokerList == nil {
- var found bool
- partitionSlotToBrokerList, found = cg.pubBalancer.TopicToBrokers.Get(cg.topic.String())
- if !found {
- glog.V(0).Infof("topic %s not found in balancer", cg.topic.String())
- return
- }
- }
-
- // collect current consumer group instance ids
- var consumerInstanceIds []string
- for _, consumerGroupInstance := range cg.ConsumerGroupInstances.Items() {
- consumerInstanceIds = append(consumerInstanceIds, consumerGroupInstance.InstanceId)
- }
-
- cg.mapping.BalanceToConsumerInstanceIds(partitionSlotToBrokerList, consumerInstanceIds)
-
- // convert cg.mapping currentMapping to map of consumer group instance id to partition slots
- consumerInstanceToPartitionSlots := make(map[string][]*PartitionSlotToConsumerInstance)
- for _, partitionSlot := range cg.mapping.currentMapping.PartitionSlots {
- consumerInstanceToPartitionSlots[partitionSlot.AssignedInstanceId] = append(consumerInstanceToPartitionSlots[partitionSlot.AssignedInstanceId], partitionSlot)
- }
-
- // notify consumer group instances
- for _, consumerGroupInstance := range cg.ConsumerGroupInstances.Items() {
- partitionSlots, found := consumerInstanceToPartitionSlots[consumerGroupInstance.InstanceId]
- if !found {
- partitionSlots = make([]*PartitionSlotToConsumerInstance, 0)
- }
- consumerGroupInstance.Partitions = ToPartitions(partitionSlotToBrokerList.RingSize, partitionSlots)
- assignedPartitions := make([]*mq_pb.BrokerPartitionAssignment, len(partitionSlots))
- for i, partitionSlot := range partitionSlots {
- assignedPartitions[i] = &mq_pb.BrokerPartitionAssignment{
- Partition: &mq_pb.Partition{
- RangeStop: partitionSlot.RangeStop,
- RangeStart: partitionSlot.RangeStart,
- RingSize: partitionSlotToBrokerList.RingSize,
- UnixTimeNs: partitionSlot.UnixTimeNs,
- },
- LeaderBroker: partitionSlot.Broker,
- }
- }
- response := &mq_pb.SubscriberToSubCoordinatorResponse{
- Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
- Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
- PartitionAssignments: assignedPartitions,
- },
- },
- }
- println("sending response to", consumerGroupInstance.InstanceId, "...")
- consumerGroupInstance.ResponseChan <- response
- }
+func (cg *ConsumerGroup) OnPartitionListChange(assignments []*mq_pb.BrokerPartitionAssignment) {
+}
+func (cg *ConsumerGroup) Shutdown() {
+ close(cg.stopCh)
}
diff --git a/weed/mq/sub_coordinator/consumer_group_instance.go b/weed/mq/sub_coordinator/consumer_group_instance.go
new file mode 100644
index 000000000..fce7dde66
--- /dev/null
+++ b/weed/mq/sub_coordinator/consumer_group_instance.go
@@ -0,0 +1,23 @@
+package sub_coordinator
+
+import (
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+)
+
+type ConsumerGroupInstanceId string
+
+type ConsumerGroupInstance struct {
+ InstanceId ConsumerGroupInstanceId
+ AssignedPartitions []topic.Partition
+ ResponseChan chan *mq_pb.SubscriberToSubCoordinatorResponse
+ MaxPartitionCount int32
+}
+
+func NewConsumerGroupInstance(instanceId string, maxPartitionCount int32) *ConsumerGroupInstance {
+ return &ConsumerGroupInstance{
+ InstanceId: ConsumerGroupInstanceId(instanceId),
+ ResponseChan: make(chan *mq_pb.SubscriberToSubCoordinatorResponse, 1),
+ MaxPartitionCount: maxPartitionCount,
+ }
+}
diff --git a/weed/mq/sub_coordinator/coordinator.go b/weed/mq/sub_coordinator/coordinator.go
deleted file mode 100644
index bb50991ab..000000000
--- a/weed/mq/sub_coordinator/coordinator.go
+++ /dev/null
@@ -1,111 +0,0 @@
-package sub_coordinator
-
-import (
- cmap "github.com/orcaman/concurrent-map/v2"
- "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
- "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
-)
-
-type TopicConsumerGroups struct {
- // map a consumer group name to a consumer group
- ConsumerGroups cmap.ConcurrentMap[string, *ConsumerGroup]
-}
-
-// Coordinator coordinates the instances in the consumer group for one topic.
-// It is responsible for:
-// 1. (Maybe) assigning partitions when a consumer instance is up/down.
-
-type Coordinator struct {
- // map topic name to consumer groups
- TopicSubscribers cmap.ConcurrentMap[string, *TopicConsumerGroups]
- balancer *pub_balancer.Balancer
-}
-
-func NewCoordinator(balancer *pub_balancer.Balancer) *Coordinator {
- return &Coordinator{
- TopicSubscribers: cmap.New[*TopicConsumerGroups](),
- balancer: balancer,
- }
-}
-
-func (c *Coordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
- topicName := toTopicName(topic)
- tcg, _ := c.TopicSubscribers.Get(topicName)
- if tcg == nil && createIfMissing {
- tcg = &TopicConsumerGroups{
- ConsumerGroups: cmap.New[*ConsumerGroup](),
- }
- if !c.TopicSubscribers.SetIfAbsent(topicName, tcg) {
- tcg, _ = c.TopicSubscribers.Get(topicName)
- }
- }
- return tcg
-}
-func (c *Coordinator) RemoveTopic(topic *mq_pb.Topic) {
- topicName := toTopicName(topic)
- c.TopicSubscribers.Remove(topicName)
-}
-
-func toTopicName(topic *mq_pb.Topic) string {
- topicName := topic.Namespace + "." + topic.Name
- return topicName
-}
-
-func (c *Coordinator) AddSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) *ConsumerGroupInstance {
- tcg := c.GetTopicConsumerGroups(topic, true)
- cg, _ := tcg.ConsumerGroups.Get(consumerGroup)
- if cg == nil {
- cg = NewConsumerGroup(topic, c.balancer)
- if !tcg.ConsumerGroups.SetIfAbsent(consumerGroup, cg) {
- cg, _ = tcg.ConsumerGroups.Get(consumerGroup)
- }
- }
- cgi, _ := cg.ConsumerGroupInstances.Get(consumerGroupInstance)
- if cgi == nil {
- cgi = NewConsumerGroupInstance(consumerGroupInstance)
- if !cg.ConsumerGroupInstances.SetIfAbsent(consumerGroupInstance, cgi) {
- cgi, _ = cg.ConsumerGroupInstances.Get(consumerGroupInstance)
- }
- }
- cg.OnAddConsumerGroupInstance(consumerGroupInstance, topic)
- return cgi
-}
-
-func (c *Coordinator) RemoveSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) {
- tcg := c.GetTopicConsumerGroups(topic, false)
- if tcg == nil {
- return
- }
- cg, _ := tcg.ConsumerGroups.Get(consumerGroup)
- if cg == nil {
- return
- }
- cg.ConsumerGroupInstances.Remove(consumerGroupInstance)
- cg.OnRemoveConsumerGroupInstance(consumerGroupInstance, topic)
- if cg.ConsumerGroupInstances.Count() == 0 {
- tcg.ConsumerGroups.Remove(consumerGroup)
- }
- if tcg.ConsumerGroups.Count() == 0 {
- c.RemoveTopic(topic)
- }
-}
-
-func (c *Coordinator) OnPartitionChange(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
- tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
- if tcg == nil {
- return
- }
- for _, cg := range tcg.ConsumerGroups.Items() {
- cg.OnPartitionListChange(assignments)
- }
-}
-
-// OnSubAddBroker is called when a broker is added to the balancer
-func (c *Coordinator) OnSubAddBroker(broker string, brokerStats *pub_balancer.BrokerStats) {
-
-}
-
-// OnSubRemoveBroker is called when a broker is removed from the balancer
-func (c *Coordinator) OnSubRemoveBroker(broker string, brokerStats *pub_balancer.BrokerStats) {
-
-}
diff --git a/weed/mq/sub_coordinator/inflight_message_tracker.go b/weed/mq/sub_coordinator/inflight_message_tracker.go
new file mode 100644
index 000000000..f1c46e06b
--- /dev/null
+++ b/weed/mq/sub_coordinator/inflight_message_tracker.go
@@ -0,0 +1,160 @@
+package sub_coordinator
+
+import (
+ "sort"
+ "sync"
+)
+
+type InflightMessageTracker struct {
+ messages map[string]int64
+ mu sync.Mutex
+ timestamps *RingBuffer
+}
+
+func NewInflightMessageTracker(capacity int) *InflightMessageTracker {
+ return &InflightMessageTracker{
+ messages: make(map[string]int64),
+ timestamps: NewRingBuffer(capacity),
+ }
+}
+
+// EnflightMessage tracks the message with the key and timestamp.
+// These messages are sent to the consumer group instances and waiting for ack.
+func (imt *InflightMessageTracker) EnflightMessage(key []byte, tsNs int64) {
+ // fmt.Printf("EnflightMessage(%s,%d)\n", string(key), tsNs)
+ imt.mu.Lock()
+ defer imt.mu.Unlock()
+ imt.messages[string(key)] = tsNs
+ imt.timestamps.EnflightTimestamp(tsNs)
+}
+
+// IsMessageAcknowledged returns true if the message has been acknowledged.
+// If the message is older than the oldest inflight messages, returns false.
+// returns false if the message is inflight.
+// Otherwise, returns false if the message is old and can be ignored.
+func (imt *InflightMessageTracker) IsMessageAcknowledged(key []byte, tsNs int64) bool {
+ imt.mu.Lock()
+ defer imt.mu.Unlock()
+
+ if tsNs <= imt.timestamps.OldestAckedTimestamp() {
+ return true
+ }
+ if tsNs > imt.timestamps.Latest() {
+ return false
+ }
+
+ if _, found := imt.messages[string(key)]; found {
+ return false
+ }
+
+ return true
+}
+
+// AcknowledgeMessage acknowledges the message with the key and timestamp.
+func (imt *InflightMessageTracker) AcknowledgeMessage(key []byte, tsNs int64) bool {
+ // fmt.Printf("AcknowledgeMessage(%s,%d)\n", string(key), tsNs)
+ imt.mu.Lock()
+ defer imt.mu.Unlock()
+ timestamp, exists := imt.messages[string(key)]
+ if !exists || timestamp != tsNs {
+ return false
+ }
+ delete(imt.messages, string(key))
+ // Remove the specific timestamp from the ring buffer.
+ imt.timestamps.AckTimestamp(tsNs)
+ return true
+}
+
+func (imt *InflightMessageTracker) GetOldestAckedTimestamp() int64 {
+ return imt.timestamps.OldestAckedTimestamp()
+}
+
+// IsInflight returns true if the message with the key is inflight.
+func (imt *InflightMessageTracker) IsInflight(key []byte) bool {
+ imt.mu.Lock()
+ defer imt.mu.Unlock()
+ _, found := imt.messages[string(key)]
+ return found
+}
+
+type TimestampStatus struct {
+ Timestamp int64
+ Acked bool
+}
+
+// RingBuffer represents a circular buffer to hold timestamps.
+type RingBuffer struct {
+ buffer []*TimestampStatus
+ head int
+ size int
+ maxTimestamp int64
+ maxAllAckedTs int64
+}
+
+// NewRingBuffer creates a new RingBuffer of the given capacity.
+func NewRingBuffer(capacity int) *RingBuffer {
+ return &RingBuffer{
+ buffer: newBuffer(capacity),
+ }
+}
+
+func newBuffer(capacity int) []*TimestampStatus {
+ buffer := make([]*TimestampStatus, capacity)
+ for i := range buffer {
+ buffer[i] = &TimestampStatus{}
+ }
+ return buffer
+}
+
+// EnflightTimestamp adds a new timestamp to the ring buffer.
+func (rb *RingBuffer) EnflightTimestamp(timestamp int64) {
+ if rb.size < len(rb.buffer) {
+ rb.size++
+ } else {
+ newBuf := newBuffer(2*len(rb.buffer))
+ for i := 0; i < rb.size; i++ {
+ newBuf[i] = rb.buffer[(rb.head+len(rb.buffer)-rb.size+i)%len(rb.buffer)]
+ }
+ rb.buffer = newBuf
+ rb.head = rb.size
+ rb.size++
+ }
+ head := rb.buffer[rb.head]
+ head.Timestamp = timestamp
+ head.Acked = false
+ rb.head = (rb.head + 1) % len(rb.buffer)
+ if timestamp > rb.maxTimestamp {
+ rb.maxTimestamp = timestamp
+ }
+}
+
+// AckTimestamp removes the specified timestamp from the ring buffer.
+func (rb *RingBuffer) AckTimestamp(timestamp int64) {
+ // Perform binary search
+ index := sort.Search(rb.size, func(i int) bool {
+ return rb.buffer[(rb.head+len(rb.buffer)-rb.size+i)%len(rb.buffer)].Timestamp >= timestamp
+ })
+ actualIndex := (rb.head + len(rb.buffer) - rb.size + index) % len(rb.buffer)
+
+ rb.buffer[actualIndex].Acked = true
+
+ // Remove all the continuously acknowledged timestamps from the buffer
+ startPos := (rb.head + len(rb.buffer) - rb.size) % len(rb.buffer)
+ for i := 0; i < len(rb.buffer) && rb.buffer[(startPos+i)%len(rb.buffer)].Acked; i++ {
+ t := rb.buffer[(startPos+i)%len(rb.buffer)]
+ if rb.maxAllAckedTs < t.Timestamp {
+ rb.size--
+ rb.maxAllAckedTs = t.Timestamp
+ }
+ }
+}
+
+// OldestAckedTimestamp returns the oldest that is already acked timestamp in the ring buffer.
+func (rb *RingBuffer) OldestAckedTimestamp() int64 {
+ return rb.maxAllAckedTs
+}
+
+// Latest returns the most recently known timestamp in the ring buffer.
+func (rb *RingBuffer) Latest() int64 {
+ return rb.maxTimestamp
+}
diff --git a/weed/mq/sub_coordinator/inflight_message_tracker_test.go b/weed/mq/sub_coordinator/inflight_message_tracker_test.go
new file mode 100644
index 000000000..5b7a1bdd8
--- /dev/null
+++ b/weed/mq/sub_coordinator/inflight_message_tracker_test.go
@@ -0,0 +1,133 @@
+package sub_coordinator
+
+import (
+ "github.com/stretchr/testify/assert"
+ "testing"
+)
+
+func TestRingBuffer(t *testing.T) {
+ // Initialize a RingBuffer with capacity 5
+ rb := NewRingBuffer(5)
+
+ // Add timestamps to the buffer
+ timestamps := []int64{100, 200, 300, 400, 500}
+ for _, ts := range timestamps {
+ rb.EnflightTimestamp(ts)
+ }
+
+ // Test Add method and buffer size
+ expectedSize := 5
+ if rb.size != expectedSize {
+ t.Errorf("Expected buffer size %d, got %d", expectedSize, rb.size)
+ }
+
+ assert.Equal(t, int64(0), rb.OldestAckedTimestamp())
+ assert.Equal(t, int64(500), rb.Latest())
+
+ rb.AckTimestamp(200)
+ assert.Equal(t, int64(0), rb.OldestAckedTimestamp())
+ rb.AckTimestamp(100)
+ assert.Equal(t, int64(200), rb.OldestAckedTimestamp())
+
+ rb.EnflightTimestamp(int64(600))
+ rb.EnflightTimestamp(int64(700))
+
+ rb.AckTimestamp(500)
+ assert.Equal(t, int64(200), rb.OldestAckedTimestamp())
+ rb.AckTimestamp(400)
+ assert.Equal(t, int64(200), rb.OldestAckedTimestamp())
+ rb.AckTimestamp(300)
+ assert.Equal(t, int64(500), rb.OldestAckedTimestamp())
+
+ assert.Equal(t, int64(700), rb.Latest())
+}
+
+func TestInflightMessageTracker(t *testing.T) {
+ // Initialize an InflightMessageTracker with capacity 5
+ tracker := NewInflightMessageTracker(5)
+
+ // Add inflight messages
+ key := []byte("1")
+ timestamp := int64(1)
+ tracker.EnflightMessage(key, timestamp)
+
+ // Test IsMessageAcknowledged method
+ isOld := tracker.IsMessageAcknowledged(key, timestamp-10)
+ if !isOld {
+ t.Error("Expected message to be old")
+ }
+
+ // Test AcknowledgeMessage method
+ acked := tracker.AcknowledgeMessage(key, timestamp)
+ if !acked {
+ t.Error("Expected message to be acked")
+ }
+ if _, exists := tracker.messages[string(key)]; exists {
+ t.Error("Expected message to be deleted after ack")
+ }
+ if tracker.timestamps.size != 0 {
+ t.Error("Expected buffer size to be 0 after ack")
+ }
+ assert.Equal(t, timestamp, tracker.GetOldestAckedTimestamp())
+}
+
+func TestInflightMessageTracker2(t *testing.T) {
+ // Initialize an InflightMessageTracker with initial capacity 1
+ tracker := NewInflightMessageTracker(1)
+
+ tracker.EnflightMessage([]byte("1"), int64(1))
+ tracker.EnflightMessage([]byte("2"), int64(2))
+ tracker.EnflightMessage([]byte("3"), int64(3))
+ tracker.EnflightMessage([]byte("4"), int64(4))
+ tracker.EnflightMessage([]byte("5"), int64(5))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("1"), int64(1)))
+ assert.Equal(t, int64(1), tracker.GetOldestAckedTimestamp())
+
+ // Test IsMessageAcknowledged method
+ isAcked := tracker.IsMessageAcknowledged([]byte("2"), int64(2))
+ if isAcked {
+ t.Error("Expected message to be not acked")
+ }
+
+ // Test AcknowledgeMessage method
+ assert.True(t, tracker.AcknowledgeMessage([]byte("2"), int64(2)))
+ assert.Equal(t, int64(2), tracker.GetOldestAckedTimestamp())
+
+}
+
+func TestInflightMessageTracker3(t *testing.T) {
+ // Initialize an InflightMessageTracker with initial capacity 1
+ tracker := NewInflightMessageTracker(1)
+
+ tracker.EnflightMessage([]byte("1"), int64(1))
+ tracker.EnflightMessage([]byte("2"), int64(2))
+ tracker.EnflightMessage([]byte("3"), int64(3))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("1"), int64(1)))
+ tracker.EnflightMessage([]byte("4"), int64(4))
+ tracker.EnflightMessage([]byte("5"), int64(5))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("2"), int64(2)))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("3"), int64(3)))
+ tracker.EnflightMessage([]byte("6"), int64(6))
+ tracker.EnflightMessage([]byte("7"), int64(7))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("4"), int64(4)))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("5"), int64(5)))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("6"), int64(6)))
+ assert.Equal(t, int64(6), tracker.GetOldestAckedTimestamp())
+ assert.True(t, tracker.AcknowledgeMessage([]byte("7"), int64(7)))
+ assert.Equal(t, int64(7), tracker.GetOldestAckedTimestamp())
+
+}
+
+func TestInflightMessageTracker4(t *testing.T) {
+ // Initialize an InflightMessageTracker with initial capacity 1
+ tracker := NewInflightMessageTracker(1)
+
+ tracker.EnflightMessage([]byte("1"), int64(1))
+ tracker.EnflightMessage([]byte("2"), int64(2))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("1"), int64(1)))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("2"), int64(2)))
+ tracker.EnflightMessage([]byte("3"), int64(3))
+ assert.True(t, tracker.AcknowledgeMessage([]byte("3"), int64(3)))
+ assert.Equal(t, int64(3), tracker.GetOldestAckedTimestamp())
+
+}
diff --git a/weed/mq/sub_coordinator/market.go b/weed/mq/sub_coordinator/market.go
new file mode 100644
index 000000000..cce6be4e1
--- /dev/null
+++ b/weed/mq/sub_coordinator/market.go
@@ -0,0 +1,367 @@
+package sub_coordinator
+
+import (
+ "errors"
+ "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "sync"
+ "time"
+)
+
+/*
+Market is a data structure that keeps track of the state of the consumer group instances and the partitions.
+
+When rebalancing, the market will try to balance the load of the partitions among the consumer group instances.
+For each loop, the market will:
+* If a consumer group instance has more partitions than the average, it will unassign some partitions.
+* If a consumer group instance has less partitions than the average, it will assign some partitions.
+
+Trigger rebalance when:
+* A new consumer group instance is added
+* Some partitions are unassigned from a consumer group instance.
+
+If multiple reblance requests are received, after a certain period, the market will only process the latest request.
+
+However, if the number of unassigned partition is increased to exactly the total number of partitions,
+and total partitions are less than or equal to the sum of the max partition count of all consumer group instances,
+the market will process the request immediately.
+This is to ensure a partition can be migrated to another consumer group instance as soon as possible.
+
+Emit these adjustments to the subscriber coordinator:
+* Assign a partition to a consumer group instance
+* Unassign a partition from a consumer group instance
+
+Because the adjustment is sent to the subscriber coordinator, the market will keep track of the inflight adjustments.
+The subscriber coordinator will send back the response to the market when the adjustment is processed.
+If the adjustment is older than a certain time(inflightAdjustmentTTL), it would be considered expired.
+Otherwise, the adjustment is considered inflight, so it would be used when calculating the load.
+
+Later features:
+* A consumer group instance is not keeping up with the load.
+
+Since a coordinator, and thus the market, may be restarted or moved to another node, the market should be able to recover the state from the subscriber coordinator.
+The subscriber coordinator should be able to send the current state of the consumer group instances and the partitions to the market.
+
+*/
+
+type PartitionSlot struct {
+ Partition topic.Partition
+ AssignedTo *ConsumerGroupInstance // Track the consumer assigned to this partition slot
+}
+
+type Adjustment struct {
+ isAssign bool
+ partition topic.Partition
+ consumer ConsumerGroupInstanceId
+ ts time.Time
+}
+
+type Market struct {
+ mu sync.Mutex
+ partitions map[topic.Partition]*PartitionSlot
+ consumerInstances map[ConsumerGroupInstanceId]*ConsumerGroupInstance
+ AdjustmentChan chan *Adjustment
+ inflightAdjustments []*Adjustment
+ inflightAdjustmentTTL time.Duration
+ lastBalancedTime time.Time
+ stopChan chan struct{}
+ balanceRequestChan chan struct{}
+ hasBalanceRequest bool
+}
+
+func NewMarket(partitions []topic.Partition, inflightAdjustmentTTL time.Duration) *Market {
+ partitionMap := make(map[topic.Partition]*PartitionSlot)
+ for _, partition := range partitions {
+ partitionMap[partition] = &PartitionSlot{
+ Partition: partition,
+ }
+ }
+ m := &Market{
+ partitions: partitionMap,
+ consumerInstances: make(map[ConsumerGroupInstanceId]*ConsumerGroupInstance),
+ AdjustmentChan: make(chan *Adjustment, 100),
+ inflightAdjustmentTTL: inflightAdjustmentTTL,
+ stopChan: make(chan struct{}),
+ balanceRequestChan: make(chan struct{}),
+ }
+ m.lastBalancedTime = time.Now()
+ go m.loopBalanceLoad()
+
+ return m
+}
+
+func (m *Market) ShutdownMarket() {
+ close(m.stopChan)
+ close(m.AdjustmentChan)
+}
+
+func (m *Market) AddConsumerInstance(consumer *ConsumerGroupInstance) error {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+
+ if _, exists := m.consumerInstances[consumer.InstanceId]; exists {
+ return errors.New("consumer instance already exists")
+ }
+
+ m.consumerInstances[consumer.InstanceId] = consumer
+ m.balanceRequestChan <- struct{}{}
+
+ return nil
+}
+
+func (m *Market) RemoveConsumerInstance(consumerId ConsumerGroupInstanceId) error {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+
+ consumer, exists := m.consumerInstances[consumerId]
+ if !exists {
+ return nil
+ }
+ delete(m.consumerInstances, consumerId)
+
+ for _, partition := range consumer.AssignedPartitions {
+ if partitionSlot, exists := m.partitions[partition]; exists {
+ partitionSlot.AssignedTo = nil
+ }
+ }
+ m.balanceRequestChan <- struct{}{}
+
+ return nil
+}
+
+func (m *Market) assignPartitionToConsumer(partition *PartitionSlot) {
+ var bestConsumer *ConsumerGroupInstance
+ var minLoad = int(^uint(0) >> 1) // Max int value
+
+ inflightConsumerAdjustments := make(map[ConsumerGroupInstanceId]int)
+ for _, adjustment := range m.inflightAdjustments {
+ if adjustment.isAssign {
+ inflightConsumerAdjustments[adjustment.consumer]++
+ } else {
+ inflightConsumerAdjustments[adjustment.consumer]--
+ }
+ }
+ for _, consumer := range m.consumerInstances {
+ consumerLoad := len(consumer.AssignedPartitions)
+ if inflightAdjustments, exists := inflightConsumerAdjustments[consumer.InstanceId]; exists {
+ consumerLoad += inflightAdjustments
+ }
+ // fmt.Printf("Consumer %+v has load %d, max %d, min %d\n", consumer.InstanceId, consumerLoad, consumer.MaxPartitionCount, minLoad)
+ if consumerLoad < int(consumer.MaxPartitionCount) {
+ if consumerLoad < minLoad {
+ bestConsumer = consumer
+ minLoad = consumerLoad
+ // fmt.Printf("picked: Consumer %+v has load %d, max %d, min %d\n", consumer.InstanceId, consumerLoad, consumer.MaxPartitionCount, minLoad)
+ }
+ }
+ }
+
+ if bestConsumer != nil {
+ // change consumer assigned partitions later when the adjustment is confirmed
+ adjustment := &Adjustment{
+ isAssign: true,
+ partition: partition.Partition,
+ consumer: bestConsumer.InstanceId,
+ ts: time.Now(),
+ }
+ m.AdjustmentChan <- adjustment
+ m.inflightAdjustments = append(m.inflightAdjustments, adjustment)
+ m.lastBalancedTime = adjustment.ts
+ }
+}
+
+func (m *Market) loopBalanceLoad() {
+ ticker := time.NewTicker(500 * time.Millisecond)
+ defer ticker.Stop()
+ for {
+ select {
+ case <-ticker.C:
+ if m.hasBalanceRequest {
+ m.hasBalanceRequest = false
+ inflightAdjustments := make([]*Adjustment, 0, len(m.inflightAdjustments))
+ for _, adjustment := range m.inflightAdjustments {
+ if adjustment.ts.Add(m.inflightAdjustmentTTL).After(time.Now()) {
+ inflightAdjustments = append(inflightAdjustments, adjustment)
+ }
+ }
+ m.inflightAdjustments = inflightAdjustments
+
+ m.doBalanceLoad()
+ // println("Balance load completed.")
+ m.Status()
+ }
+ case <-m.balanceRequestChan:
+ m.hasBalanceRequest = true
+ case <-m.stopChan:
+ return
+ }
+ }
+}
+
+// doBalanceLoad will balance the load of the partitions among the consumer group instances.
+// It will try to unassign partitions from the consumer group instances that have more partitions than the average.
+// It will try to assign partitions to the consumer group instances that have less partitions than the average.
+func (m *Market) doBalanceLoad() {
+ if len(m.consumerInstances) == 0 {
+ return
+ }
+
+ // find the average load for all consumers
+ averageLoad := m.findAverageLoad()
+
+ // find the consumers with the higher load than average
+ if m.adjustBusyConsumers(averageLoad) {
+ return
+ }
+
+ // find partitions with no consumer assigned
+ m.adjustUnassignedPartitions()
+}
+func (m *Market) findAverageLoad() (averageLoad float32) {
+ var totalLoad int
+ for _, consumer := range m.consumerInstances {
+ totalLoad += len(consumer.AssignedPartitions)
+ }
+ for _, adjustment := range m.inflightAdjustments {
+ if adjustment.isAssign {
+ totalLoad++
+ } else {
+ totalLoad--
+ }
+ }
+ averageLoad = float32(totalLoad) / float32(len(m.consumerInstances))
+ return
+}
+
+func (m *Market) adjustBusyConsumers(averageLoad float32) (hasAdjustments bool) {
+ inflightConsumerAdjustments := make(map[ConsumerGroupInstanceId]int)
+ for _, adjustment := range m.inflightAdjustments {
+ if adjustment.isAssign {
+ inflightConsumerAdjustments[adjustment.consumer]++
+ } else {
+ inflightConsumerAdjustments[adjustment.consumer]--
+ }
+ }
+ for _, consumer := range m.consumerInstances {
+ consumerLoad := len(consumer.AssignedPartitions)
+ if inflightAdjustment, exists := inflightConsumerAdjustments[consumer.InstanceId]; exists {
+ consumerLoad += inflightAdjustment
+ }
+ delta := int(float32(consumerLoad) - averageLoad)
+ if delta <= 0 {
+ continue
+ }
+ adjustTime := time.Now()
+ for i := 0; i < delta; i++ {
+ adjustment := &Adjustment{
+ isAssign: false,
+ partition: consumer.AssignedPartitions[i],
+ consumer: consumer.InstanceId,
+ ts: adjustTime,
+ }
+ m.AdjustmentChan <- adjustment
+ m.inflightAdjustments = append(m.inflightAdjustments, adjustment)
+ m.lastBalancedTime = adjustment.ts
+ }
+ hasAdjustments = true
+ }
+ return
+}
+
+func (m *Market) adjustUnassignedPartitions() {
+ inflightPartitionAdjustments := make(map[topic.Partition]bool)
+ for _, adjustment := range m.inflightAdjustments {
+ inflightPartitionAdjustments[adjustment.partition] = true
+ }
+ for _, partitionSlot := range m.partitions {
+ if partitionSlot.AssignedTo == nil {
+ if _, exists := inflightPartitionAdjustments[partitionSlot.Partition]; exists {
+ continue
+ }
+ // fmt.Printf("Assigning partition %+v to consumer\n", partitionSlot.Partition)
+ m.assignPartitionToConsumer(partitionSlot)
+ }
+ }
+}
+
+func (m *Market) ConfirmAdjustment(adjustment *Adjustment) {
+ if adjustment.isAssign {
+ m.confirmAssignPartition(adjustment.partition, adjustment.consumer)
+ } else {
+ m.unassignPartitionSlot(adjustment.partition)
+ }
+ glog.V(1).Infof("ConfirmAdjustment %+v", adjustment)
+ m.Status()
+}
+
+func (m *Market) unassignPartitionSlot(partition topic.Partition) {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+
+ partitionSlot, exists := m.partitions[partition]
+ if !exists {
+ glog.V(0).Infof("partition %+v slot is not tracked", partition)
+ return
+ }
+
+ if partitionSlot.AssignedTo == nil {
+ glog.V(0).Infof("partition %+v slot is not assigned to any consumer", partition)
+ return
+ }
+
+ consumer := partitionSlot.AssignedTo
+ for i, p := range consumer.AssignedPartitions {
+ if p == partition {
+ consumer.AssignedPartitions = append(consumer.AssignedPartitions[:i], consumer.AssignedPartitions[i+1:]...)
+ partitionSlot.AssignedTo = nil
+ m.balanceRequestChan <- struct{}{}
+ return
+ }
+ }
+
+ glog.V(0).Infof("partition %+v slot not found in assigned consumer", partition)
+
+}
+
+func (m *Market) confirmAssignPartition(partition topic.Partition, consumerInstanceId ConsumerGroupInstanceId) {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+
+ partitionSlot, exists := m.partitions[partition]
+ if !exists {
+ glog.V(0).Infof("partition %+v slot is not tracked", partition)
+ return
+ }
+
+ if partitionSlot.AssignedTo != nil {
+ glog.V(0).Infof("partition %+v slot is already assigned to %+v", partition, partitionSlot.AssignedTo.InstanceId)
+ return
+ }
+
+ consumerInstance, exists := m.consumerInstances[consumerInstanceId]
+ if !exists {
+ glog.V(0).Infof("consumer %+v is not tracked", consumerInstanceId)
+ return
+ }
+
+ partitionSlot.AssignedTo = consumerInstance
+ consumerInstance.AssignedPartitions = append(consumerInstance.AssignedPartitions, partition)
+
+}
+
+func (m *Market) Status() {
+ m.mu.Lock()
+ defer m.mu.Unlock()
+
+ glog.V(1).Infof("Market has %d partitions and %d consumer instances", len(m.partitions), len(m.consumerInstances))
+ for partition, slot := range m.partitions {
+ if slot.AssignedTo == nil {
+ glog.V(1).Infof("Partition %+v is not assigned to any consumer", partition)
+ } else {
+ glog.V(1).Infof("Partition %+v is assigned to consumer %+v", partition, slot.AssignedTo.InstanceId)
+ }
+ }
+ for _, consumer := range m.consumerInstances {
+ glog.V(1).Infof("Consumer %+v has %d partitions", consumer.InstanceId, len(consumer.AssignedPartitions))
+ }
+}
diff --git a/weed/mq/sub_coordinator/market_test.go b/weed/mq/sub_coordinator/market_test.go
new file mode 100644
index 000000000..150a88a8d
--- /dev/null
+++ b/weed/mq/sub_coordinator/market_test.go
@@ -0,0 +1,103 @@
+package sub_coordinator
+
+import (
+ "fmt"
+ "testing"
+ "time"
+
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
+ "github.com/stretchr/testify/assert"
+)
+
+var partitions = []topic.Partition{
+ {
+ RangeStart: 0,
+ RangeStop: 1,
+ RingSize: 3,
+ UnixTimeNs: 0,
+ },
+ {
+ RangeStart: 1,
+ RangeStop: 2,
+ RingSize: 3,
+ UnixTimeNs: 0,
+ },
+ {
+ RangeStart: 2,
+ RangeStop: 3,
+ RingSize: 3,
+ UnixTimeNs: 0,
+ },
+}
+
+func TestAddConsumerInstance(t *testing.T) {
+ market := NewMarket(partitions, 10*time.Second)
+
+ consumer := &ConsumerGroupInstance{
+ InstanceId: "first",
+ MaxPartitionCount: 2,
+ }
+ err := market.AddConsumerInstance(consumer)
+
+ assert.Nil(t, err)
+ time.Sleep(1 * time.Second) // Allow time for background rebalancing
+ market.ShutdownMarket()
+ for adjustment := range market.AdjustmentChan {
+ fmt.Printf("%+v\n", adjustment)
+ }
+}
+
+func TestMultipleConsumerInstances(t *testing.T) {
+ market := NewMarket(partitions, 10*time.Second)
+
+ market.AddConsumerInstance(&ConsumerGroupInstance{
+ InstanceId: "first",
+ MaxPartitionCount: 2,
+ })
+ market.AddConsumerInstance(&ConsumerGroupInstance{
+ InstanceId: "second",
+ MaxPartitionCount: 2,
+ })
+ market.AddConsumerInstance(&ConsumerGroupInstance{
+ InstanceId: "third",
+ MaxPartitionCount: 2,
+ })
+
+ time.Sleep(1 * time.Second) // Allow time for background rebalancing
+ market.ShutdownMarket()
+ for adjustment := range market.AdjustmentChan {
+ fmt.Printf("%+v\n", adjustment)
+ }
+}
+
+func TestConfirmAdjustment(t *testing.T) {
+ market := NewMarket(partitions, 1*time.Second)
+
+ market.AddConsumerInstance(&ConsumerGroupInstance{
+ InstanceId: "first",
+ MaxPartitionCount: 2,
+ })
+ market.AddConsumerInstance(&ConsumerGroupInstance{
+ InstanceId: "second",
+ MaxPartitionCount: 2,
+ })
+ market.AddConsumerInstance(&ConsumerGroupInstance{
+ InstanceId: "third",
+ MaxPartitionCount: 2,
+ })
+
+ go func() {
+ time.Sleep(5 * time.Second) // Allow time for background rebalancing
+ market.ShutdownMarket()
+ }()
+ go func() {
+ time.Sleep(2 * time.Second)
+ market.RemoveConsumerInstance("third")
+ }()
+
+ for adjustment := range market.AdjustmentChan {
+ fmt.Printf("%+v\n", adjustment)
+ market.ConfirmAdjustment(adjustment)
+ }
+
+}
diff --git a/weed/mq/sub_coordinator/partition_consumer_mapping.go b/weed/mq/sub_coordinator/partition_consumer_mapping.go
index c7f104af1..5d1cf158a 100644
--- a/weed/mq/sub_coordinator/partition_consumer_mapping.go
+++ b/weed/mq/sub_coordinator/partition_consumer_mapping.go
@@ -23,8 +23,8 @@ func NewPartitionConsumerMapping(ringSize int32) *PartitionConsumerMapping {
// 2. allow one consumer instance to be down unexpectedly
// without affecting the processing power utilization
-func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitionSlotToBrokerList *pub_balancer.PartitionSlotToBrokerList, consumerInstanceIds []string) {
- if len(partitionSlotToBrokerList.PartitionSlots) == 0 || len(consumerInstanceIds) == 0 {
+func (pcm *PartitionConsumerMapping) BalanceToConsumerInstances(partitionSlotToBrokerList *pub_balancer.PartitionSlotToBrokerList, consumerInstances []*ConsumerGroupInstance) {
+ if len(partitionSlotToBrokerList.PartitionSlots) == 0 || len(consumerInstances) == 0 {
return
}
newVersion := time.Now().UnixNano()
@@ -35,7 +35,7 @@ func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitionSlotT
} else {
prevMapping = nil
}
- newMapping.PartitionSlots = doBalanceSticky(partitionSlotToBrokerList.PartitionSlots, consumerInstanceIds, prevMapping)
+ newMapping.PartitionSlots = doBalanceSticky(partitionSlotToBrokerList.PartitionSlots, consumerInstances, prevMapping)
if pcm.currentMapping != nil {
pcm.prevMappings = append(pcm.prevMappings, pcm.currentMapping)
if len(pcm.prevMappings) > 10 {
@@ -45,9 +45,9 @@ func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitionSlotT
pcm.currentMapping = newMapping
}
-func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerInstanceIds []string, prevMapping *PartitionSlotToConsumerInstanceList) (partitionSlots []*PartitionSlotToConsumerInstance) {
+func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerInstances []*ConsumerGroupInstance, prevMapping *PartitionSlotToConsumerInstanceList) (partitionSlots []*PartitionSlotToConsumerInstance) {
// collect previous consumer instance ids
- prevConsumerInstanceIds := make(map[string]struct{})
+ prevConsumerInstanceIds := make(map[ConsumerGroupInstanceId]struct{})
if prevMapping != nil {
for _, prevPartitionSlot := range prevMapping.PartitionSlots {
if prevPartitionSlot.AssignedInstanceId != "" {
@@ -56,13 +56,13 @@ func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerI
}
}
// collect current consumer instance ids
- currConsumerInstanceIds := make(map[string]struct{})
- for _, consumerInstanceId := range consumerInstanceIds {
- currConsumerInstanceIds[consumerInstanceId] = struct{}{}
+ currConsumerInstanceIds := make(map[ConsumerGroupInstanceId]struct{})
+ for _, consumerInstance := range consumerInstances {
+ currConsumerInstanceIds[consumerInstance.InstanceId] = struct{}{}
}
// check deleted consumer instances
- deletedConsumerInstanceIds := make(map[string]struct{})
+ deletedConsumerInstanceIds := make(map[ConsumerGroupInstanceId]struct{})
for consumerInstanceId := range prevConsumerInstanceIds {
if _, ok := currConsumerInstanceIds[consumerInstanceId]; !ok {
deletedConsumerInstanceIds[consumerInstanceId] = struct{}{}
@@ -82,10 +82,11 @@ func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerI
newPartitionSlots := make([]*PartitionSlotToConsumerInstance, 0, len(partitions))
for _, partition := range partitions {
newPartitionSlots = append(newPartitionSlots, &PartitionSlotToConsumerInstance{
- RangeStart: partition.RangeStart,
- RangeStop: partition.RangeStop,
- UnixTimeNs: partition.UnixTimeNs,
- Broker: partition.AssignedBroker,
+ RangeStart: partition.RangeStart,
+ RangeStop: partition.RangeStop,
+ UnixTimeNs: partition.UnixTimeNs,
+ Broker: partition.AssignedBroker,
+ FollowerBroker: partition.FollowerBroker,
})
}
for _, newPartitionSlot := range newPartitionSlots {
@@ -99,32 +100,32 @@ func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerI
// for all consumer instances, count the average number of partitions
// that are assigned to them
- consumerInstancePartitionCount := make(map[string]int)
+ consumerInstancePartitionCount := make(map[ConsumerGroupInstanceId]int)
for _, newPartitionSlot := range newPartitionSlots {
if newPartitionSlot.AssignedInstanceId != "" {
consumerInstancePartitionCount[newPartitionSlot.AssignedInstanceId]++
}
}
// average number of partitions that are assigned to each consumer instance
- averageConsumerInstanceLoad := float32(len(partitions)) / float32(len(consumerInstanceIds))
+ averageConsumerInstanceLoad := float32(len(partitions)) / float32(len(consumerInstances))
// assign unassigned partition slots to consumer instances that is underloaded
consumerInstanceIdsIndex := 0
for _, newPartitionSlot := range newPartitionSlots {
if newPartitionSlot.AssignedInstanceId == "" {
- for avoidDeadLoop := len(consumerInstanceIds); avoidDeadLoop > 0; avoidDeadLoop-- {
- consumerInstanceId := consumerInstanceIds[consumerInstanceIdsIndex]
- if float32(consumerInstancePartitionCount[consumerInstanceId]) < averageConsumerInstanceLoad {
- newPartitionSlot.AssignedInstanceId = consumerInstanceId
- consumerInstancePartitionCount[consumerInstanceId]++
+ for avoidDeadLoop := len(consumerInstances); avoidDeadLoop > 0; avoidDeadLoop-- {
+ consumerInstance := consumerInstances[consumerInstanceIdsIndex]
+ if float32(consumerInstancePartitionCount[consumerInstance.InstanceId]) < averageConsumerInstanceLoad {
+ newPartitionSlot.AssignedInstanceId = consumerInstance.InstanceId
+ consumerInstancePartitionCount[consumerInstance.InstanceId]++
consumerInstanceIdsIndex++
- if consumerInstanceIdsIndex >= len(consumerInstanceIds) {
+ if consumerInstanceIdsIndex >= len(consumerInstances) {
consumerInstanceIdsIndex = 0
}
break
} else {
consumerInstanceIdsIndex++
- if consumerInstanceIdsIndex >= len(consumerInstanceIds) {
+ if consumerInstanceIdsIndex >= len(consumerInstances) {
consumerInstanceIdsIndex = 0
}
}
diff --git a/weed/mq/sub_coordinator/partition_consumer_mapping_test.go b/weed/mq/sub_coordinator/partition_consumer_mapping_test.go
index 9a9abe011..415eb27bd 100644
--- a/weed/mq/sub_coordinator/partition_consumer_mapping_test.go
+++ b/weed/mq/sub_coordinator/partition_consumer_mapping_test.go
@@ -9,7 +9,7 @@ import (
func Test_doBalanceSticky(t *testing.T) {
type args struct {
partitions []*pub_balancer.PartitionSlotToBroker
- consumerInstanceIds []string
+ consumerInstanceIds []*ConsumerGroupInstance
prevMapping *PartitionSlotToConsumerInstanceList
}
tests := []struct {
@@ -26,8 +26,13 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 100,
},
},
- consumerInstanceIds: []string{"consumer-instance-1"},
- prevMapping: nil,
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ },
+ prevMapping: nil,
},
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -46,8 +51,17 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 100,
},
},
- consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
- prevMapping: nil,
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-2",
+ MaxPartitionCount: 1,
+ },
+ },
+ prevMapping: nil,
},
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -70,8 +84,13 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 100,
},
},
- consumerInstanceIds: []string{"consumer-instance-1"},
- prevMapping: nil,
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ },
+ prevMapping: nil,
},
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -99,8 +118,17 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 100,
},
},
- consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
- prevMapping: nil,
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-2",
+ MaxPartitionCount: 1,
+ },
+ },
+ prevMapping: nil,
},
wantPartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -128,7 +156,16 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 100,
},
},
- consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-2",
+ MaxPartitionCount: 1,
+ },
+ },
prevMapping: &PartitionSlotToConsumerInstanceList{
PartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -170,7 +207,20 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 100,
},
},
- consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2", "consumer-instance-3"},
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-2",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-3",
+ MaxPartitionCount: 1,
+ },
+ },
prevMapping: &PartitionSlotToConsumerInstanceList{
PartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -216,7 +266,16 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 150,
},
},
- consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2"},
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-2",
+ MaxPartitionCount: 1,
+ },
+ },
prevMapping: &PartitionSlotToConsumerInstanceList{
PartitionSlots: []*PartitionSlotToConsumerInstance{
{
@@ -267,7 +326,20 @@ func Test_doBalanceSticky(t *testing.T) {
RangeStop: 150,
},
},
- consumerInstanceIds: []string{"consumer-instance-1", "consumer-instance-2", "consumer-instance-3"},
+ consumerInstanceIds: []*ConsumerGroupInstance{
+ {
+ InstanceId: "consumer-instance-1",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-2",
+ MaxPartitionCount: 1,
+ },
+ {
+ InstanceId: "consumer-instance-3",
+ MaxPartitionCount: 1,
+ },
+ },
prevMapping: &PartitionSlotToConsumerInstanceList{
PartitionSlots: []*PartitionSlotToConsumerInstance{
{
diff --git a/weed/mq/sub_coordinator/partition_list.go b/weed/mq/sub_coordinator/partition_list.go
index fa0e3761f..384c1b875 100644
--- a/weed/mq/sub_coordinator/partition_list.go
+++ b/weed/mq/sub_coordinator/partition_list.go
@@ -7,7 +7,8 @@ type PartitionSlotToConsumerInstance struct {
RangeStop int32
UnixTimeNs int64
Broker string
- AssignedInstanceId string
+ AssignedInstanceId ConsumerGroupInstanceId
+ FollowerBroker string
}
type PartitionSlotToConsumerInstanceList struct {
diff --git a/weed/mq/sub_coordinator/sub_coordinator.go b/weed/mq/sub_coordinator/sub_coordinator.go
new file mode 100644
index 000000000..47101eee4
--- /dev/null
+++ b/weed/mq/sub_coordinator/sub_coordinator.go
@@ -0,0 +1,107 @@
+package sub_coordinator
+
+import (
+ "fmt"
+ cmap "github.com/orcaman/concurrent-map/v2"
+ "github.com/seaweedfs/seaweedfs/weed/filer_client"
+ "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+)
+
+type TopicConsumerGroups struct {
+ // map a consumer group name to a consumer group
+ ConsumerGroups cmap.ConcurrentMap[string, *ConsumerGroup]
+}
+
+// SubCoordinator coordinates the instances in the consumer group for one topic.
+// It is responsible for:
+// 1. (Maybe) assigning partitions when a consumer instance is up/down.
+
+type SubCoordinator struct {
+ // map topic name to consumer groups
+ TopicSubscribers cmap.ConcurrentMap[string, *TopicConsumerGroups]
+ FilerClientAccessor *filer_client.FilerClientAccessor
+}
+
+func NewSubCoordinator() *SubCoordinator {
+ return &SubCoordinator{
+ TopicSubscribers: cmap.New[*TopicConsumerGroups](),
+ }
+}
+
+func (c *SubCoordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
+ topicName := toTopicName(topic)
+ tcg, _ := c.TopicSubscribers.Get(topicName)
+ if tcg == nil && createIfMissing {
+ tcg = &TopicConsumerGroups{
+ ConsumerGroups: cmap.New[*ConsumerGroup](),
+ }
+ if !c.TopicSubscribers.SetIfAbsent(topicName, tcg) {
+ tcg, _ = c.TopicSubscribers.Get(topicName)
+ }
+ }
+ return tcg
+}
+func (c *SubCoordinator) RemoveTopic(topic *mq_pb.Topic) {
+ topicName := toTopicName(topic)
+ c.TopicSubscribers.Remove(topicName)
+}
+
+func toTopicName(topic *mq_pb.Topic) string {
+ topicName := topic.Namespace + "." + topic.Name
+ return topicName
+}
+
+func (c *SubCoordinator) AddSubscriber(initMessage *mq_pb.SubscriberToSubCoordinatorRequest_InitMessage) (*ConsumerGroup, *ConsumerGroupInstance, error) {
+ tcg := c.GetTopicConsumerGroups(initMessage.Topic, true)
+ cg, _ := tcg.ConsumerGroups.Get(initMessage.ConsumerGroup)
+ if cg == nil {
+ cg = NewConsumerGroup(initMessage.Topic, initMessage.RebalanceSeconds, c.FilerClientAccessor)
+ if cg != nil {
+ tcg.ConsumerGroups.SetIfAbsent(initMessage.ConsumerGroup, cg)
+ }
+ cg, _ = tcg.ConsumerGroups.Get(initMessage.ConsumerGroup)
+ }
+ if cg == nil {
+ return nil, nil, fmt.Errorf("fail to create consumer group %s: topic %s not found", initMessage.ConsumerGroup, initMessage.Topic)
+ }
+ cgi, _ := cg.ConsumerGroupInstances.Get(initMessage.ConsumerGroupInstanceId)
+ if cgi == nil {
+ cgi = NewConsumerGroupInstance(initMessage.ConsumerGroupInstanceId, initMessage.MaxPartitionCount)
+ if !cg.ConsumerGroupInstances.SetIfAbsent(initMessage.ConsumerGroupInstanceId, cgi) {
+ cgi, _ = cg.ConsumerGroupInstances.Get(initMessage.ConsumerGroupInstanceId)
+ }
+ }
+ cgi.MaxPartitionCount = initMessage.MaxPartitionCount
+ cg.Market.AddConsumerInstance(cgi)
+ return cg, cgi, nil
+}
+
+func (c *SubCoordinator) RemoveSubscriber(initMessage *mq_pb.SubscriberToSubCoordinatorRequest_InitMessage) {
+ tcg := c.GetTopicConsumerGroups(initMessage.Topic, false)
+ if tcg == nil {
+ return
+ }
+ cg, _ := tcg.ConsumerGroups.Get(initMessage.ConsumerGroup)
+ if cg == nil {
+ return
+ }
+ cg.ConsumerGroupInstances.Remove(initMessage.ConsumerGroupInstanceId)
+ cg.Market.RemoveConsumerInstance(ConsumerGroupInstanceId(initMessage.ConsumerGroupInstanceId))
+ if cg.ConsumerGroupInstances.Count() == 0 {
+ tcg.ConsumerGroups.Remove(initMessage.ConsumerGroup)
+ cg.Shutdown()
+ }
+ if tcg.ConsumerGroups.Count() == 0 {
+ c.RemoveTopic(initMessage.Topic)
+ }
+}
+
+func (c *SubCoordinator) OnPartitionChange(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
+ tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
+ if tcg == nil {
+ return
+ }
+ for _, cg := range tcg.ConsumerGroups.Items() {
+ cg.OnPartitionListChange(assignments)
+ }
+}
diff --git a/weed/mq/topic/local_manager.go b/weed/mq/topic/local_manager.go
index 79a84561c..44e629fc1 100644
--- a/weed/mq/topic/local_manager.go
+++ b/weed/mq/topic/local_manager.go
@@ -99,6 +99,7 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
Partition: localPartition.Partition.ToPbPartition(),
PublisherCount: int32(localPartition.Publishers.Size()),
SubscriberCount: int32(localPartition.Subscribers.Size()),
+ Follower: localPartition.Follower,
}
// fmt.Printf("collect topic %+v partition %+v\n", topicPartition, localPartition.Partition)
}
diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go
index 72e78d606..8911c1841 100644
--- a/weed/mq/topic/local_partition.go
+++ b/weed/mq/topic/local_partition.go
@@ -28,9 +28,9 @@ type LocalPartition struct {
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
- followerStream mq_pb.SeaweedMessaging_PublishFollowMeClient
+ publishFolloweMeStream mq_pb.SeaweedMessaging_PublishFollowMeClient
followerGrpcConnection *grpc.ClientConn
- follower string
+ Follower string
}
var TIME_FORMAT = "2006-01-02-15-04-05"
@@ -55,14 +55,14 @@ func (p *LocalPartition) Publish(message *mq_pb.DataMessage) error {
p.LogBuffer.AddToBuffer(message)
// maybe send to the follower
- if p.followerStream != nil {
+ if p.publishFolloweMeStream != nil {
// println("recv", string(message.Key), message.TsNs)
- if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
+ if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Data{
Data: message,
},
}); followErr != nil {
- return fmt.Errorf("send to follower %s: %v", p.follower, followErr)
+ return fmt.Errorf("send to follower %s: %v", p.Follower, followErr)
}
} else {
atomic.StoreInt64(&p.AckTsNs, message.TsNs)
@@ -134,25 +134,25 @@ func (p *LocalPartition) WaitUntilNoPublishers() {
}
func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessageRequest_InitMessage, grpcDialOption grpc.DialOption) (err error) {
- if p.followerStream != nil {
+ if p.publishFolloweMeStream != nil {
return nil
}
- if len(initMessage.FollowerBrokers) == 0 {
+ if initMessage.FollowerBroker == "" {
return nil
}
- p.follower = initMessage.FollowerBrokers[0]
+ p.Follower = initMessage.FollowerBroker
ctx := context.Background()
- p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.follower, true, grpcDialOption)
+ p.followerGrpcConnection, err = pb.GrpcDial(ctx, p.Follower, true, grpcDialOption)
if err != nil {
- return fmt.Errorf("fail to dial %s: %v", p.follower, err)
+ return fmt.Errorf("fail to dial %s: %v", p.Follower, err)
}
followerClient := mq_pb.NewSeaweedMessagingClient(p.followerGrpcConnection)
- p.followerStream, err = followerClient.PublishFollowMe(ctx)
+ p.publishFolloweMeStream, err = followerClient.PublishFollowMe(ctx)
if err != nil {
return fmt.Errorf("fail to create publish client: %v", err)
}
- if err = p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
+ if err = p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Init{
Init: &mq_pb.PublishFollowMeRequest_InitMessage{
Topic: initMessage.Topic,
@@ -170,14 +170,14 @@ func (p *LocalPartition) MaybeConnectToFollowers(initMessage *mq_pb.PublishMessa
}()
for {
- ack, err := p.followerStream.Recv()
+ ack, err := p.publishFolloweMeStream.Recv()
if err != nil {
e, _ := status.FromError(err)
if e.Code() == codes.Canceled {
- glog.V(0).Infof("local partition %v follower %v stopped", p.Partition, p.follower)
+ glog.V(0).Infof("local partition %v follower %v stopped", p.Partition, p.Follower)
return
}
- glog.Errorf("Receiving local partition %v follower %s ack: %v", p.Partition, p.follower, err)
+ glog.Errorf("Receiving local partition %v follower %s ack: %v", p.Partition, p.Follower, err)
return
}
atomic.StoreInt64(&p.AckTsNs, ack.AckTsNs)
@@ -194,9 +194,9 @@ func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
for !p.LogBuffer.IsAllFlushed() {
time.Sleep(113 * time.Millisecond)
}
- if p.followerStream != nil {
+ if p.publishFolloweMeStream != nil {
// send close to the follower
- if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
+ if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Close{
Close: &mq_pb.PublishFollowMeRequest_CloseMessage{},
},
@@ -205,14 +205,14 @@ func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
}
glog.V(4).Infof("closing grpcConnection to follower")
p.followerGrpcConnection.Close()
- p.followerStream = nil
- p.follower = ""
+ p.publishFolloweMeStream = nil
+ p.Follower = ""
}
hasShutdown = true
}
- glog.V(0).Infof("local partition %v Publisher:%d Subscriber:%d follower:%s shutdown %v", p.Partition, p.Publishers.Size(), p.Subscribers.Size(), p.follower, hasShutdown)
+ glog.V(0).Infof("local partition %v Publisher:%d Subscriber:%d follower:%s shutdown %v", p.Partition, p.Publishers.Size(), p.Subscribers.Size(), p.Follower, hasShutdown)
return
}
@@ -224,16 +224,16 @@ func (p *LocalPartition) Shutdown() {
}
func (p *LocalPartition) NotifyLogFlushed(flushTsNs int64) {
- if p.followerStream != nil {
- if followErr := p.followerStream.Send(&mq_pb.PublishFollowMeRequest{
+ if p.publishFolloweMeStream != nil {
+ if followErr := p.publishFolloweMeStream.Send(&mq_pb.PublishFollowMeRequest{
Message: &mq_pb.PublishFollowMeRequest_Flush{
Flush: &mq_pb.PublishFollowMeRequest_FlushMessage{
TsNs: flushTsNs,
},
},
}); followErr != nil {
- glog.Errorf("send follower %s flush message: %v", p.follower, followErr)
+ glog.Errorf("send follower %s flush message: %v", p.Follower, followErr)
}
- // println("notifying", p.follower, "flushed at", flushTsNs)
+ // println("notifying", p.Follower, "flushed at", flushTsNs)
}
}
diff --git a/weed/mq/topic/partition.go b/weed/mq/topic/partition.go
index 45b55c43b..ba1accce1 100644
--- a/weed/mq/topic/partition.go
+++ b/weed/mq/topic/partition.go
@@ -71,3 +71,13 @@ func (partition Partition) ToPbPartition() *mq_pb.Partition {
UnixTimeNs: partition.UnixTimeNs,
}
}
+
+func (partition Partition) Overlaps(partition2 Partition) bool {
+ if partition.RangeStart >= partition2.RangeStop {
+ return false
+ }
+ if partition.RangeStop <= partition2.RangeStart {
+ return false
+ }
+ return true
+}
diff --git a/weed/pb/mq.proto b/weed/pb/mq.proto
index 9e4874f30..23284b767 100644
--- a/weed/pb/mq.proto
+++ b/weed/pb/mq.proto
@@ -45,11 +45,13 @@ service SeaweedMessaging {
// data plane for each topic partition
rpc PublishMessage (stream PublishMessageRequest) returns (stream PublishMessageResponse) {
}
- rpc SubscribeMessage (SubscribeMessageRequest) returns (stream SubscribeMessageResponse) {
+ rpc SubscribeMessage (stream SubscribeMessageRequest) returns (stream SubscribeMessageResponse) {
}
// The lead broker asks a follower broker to follow itself
rpc PublishFollowMe (stream PublishFollowMeRequest) returns (stream PublishFollowMeResponse) {
}
+ rpc SubscribeFollowMe (stream SubscribeFollowMeRequest) returns (SubscribeFollowMeResponse) {
+ }
}
//////////////////////////////////////////////////
@@ -101,6 +103,7 @@ message TopicPartitionStats {
Partition partition = 2;
int32 publisher_count = 3;
int32 subscriber_count = 4;
+ string follower = 5;
}
@@ -146,7 +149,7 @@ message LookupTopicBrokersResponse {
message BrokerPartitionAssignment {
Partition partition = 1;
string leader_broker = 2;
- repeated string follower_brokers = 3;
+ string follower_broker = 3;
}
message AssignTopicPartitionsRequest {
@@ -163,29 +166,46 @@ message SubscriberToSubCoordinatorRequest {
string consumer_group = 1;
string consumer_group_instance_id = 2;
Topic topic = 3;
+ // The consumer group instance will be assigned at most max_partition_count partitions.
+ // If the number of partitions is less than the sum of max_partition_count,
+ // the consumer group instance may be assigned partitions less than max_partition_count.
+ // Default is 1.
+ int32 max_partition_count = 4;
+ // If consumer group instance changes, wait for rebalance_seconds before reassigning partitions
+ // Exception: if adding a new consumer group instance and sum of max_partition_count equals the number of partitions,
+ // the rebalance will happen immediately.
+ // Default is 10 seconds.
+ int32 rebalance_seconds = 5;
}
- message AckMessage {
+ message AckUnAssignmentMessage {
+ Partition partition = 1;
+ }
+ message AckAssignmentMessage {
Partition partition = 1;
- int64 ts_ns = 2;
}
oneof message {
InitMessage init = 1;
- AckMessage ack = 2;
+ AckAssignmentMessage ack_assignment = 2;
+ AckUnAssignmentMessage ack_un_assignment = 3;
}
}
message SubscriberToSubCoordinatorResponse {
message Assignment {
- int64 generation = 1;
- repeated BrokerPartitionAssignment partition_assignments = 2;
+ BrokerPartitionAssignment partition_assignment = 1;
+ }
+ message UnAssignment {
+ Partition partition = 1;
}
oneof message {
Assignment assignment = 1;
+ UnAssignment un_assignment = 2;
}
}
//////////////////////////////////////////////////
message ControlMessage {
bool is_close = 1;
+ string publisher_name = 2;
}
message DataMessage {
bytes key = 1;
@@ -198,14 +218,13 @@ message PublishMessageRequest {
Topic topic = 1;
Partition partition = 2;
int32 ack_interval = 3;
- repeated string follower_brokers = 4;
+ string follower_broker = 4;
string publisher_name = 5; // for debugging
}
oneof message {
InitMessage init = 1;
DataMessage data = 2;
}
- int64 sequence = 3;
}
message PublishMessageResponse {
int64 ack_sequence = 1;
@@ -240,10 +259,12 @@ message SubscribeMessageRequest {
Topic topic = 4;
PartitionOffset partition_offset = 5;
string filter = 6;
- repeated string follower_brokers = 7;
+ string follower_broker = 7;
+ int32 concurrency = 8;
}
message AckMessage {
int64 sequence = 1;
+ bytes key = 2;
}
oneof message {
InitMessage init = 1;
@@ -261,6 +282,26 @@ message SubscribeMessageResponse {
DataMessage data = 2;
}
}
+message SubscribeFollowMeRequest {
+ message InitMessage {
+ Topic topic = 1;
+ Partition partition = 2;
+ string consumer_group = 3;
+ }
+ message AckMessage {
+ int64 ts_ns = 1;
+ }
+ message CloseMessage {
+ }
+ oneof message {
+ InitMessage init = 1;
+ AckMessage ack = 2;
+ CloseMessage close = 3;
+ }
+}
+message SubscribeFollowMeResponse {
+ int64 ack_ts_ns = 1;
+}
message ClosePublishersRequest {
Topic topic = 1;
int64 unix_time_ns = 2;
diff --git a/weed/pb/mq_pb/mq.pb.go b/weed/pb/mq_pb/mq.pb.go
index ff381aec9..fb76a1f50 100644
--- a/weed/pb/mq_pb/mq.pb.go
+++ b/weed/pb/mq_pb/mq.pb.go
@@ -481,6 +481,7 @@ type TopicPartitionStats struct {
Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"`
PublisherCount int32 `protobuf:"varint,3,opt,name=publisher_count,json=publisherCount,proto3" json:"publisher_count,omitempty"`
SubscriberCount int32 `protobuf:"varint,4,opt,name=subscriber_count,json=subscriberCount,proto3" json:"subscriber_count,omitempty"`
+ Follower string `protobuf:"bytes,5,opt,name=follower,proto3" json:"follower,omitempty"`
}
func (x *TopicPartitionStats) Reset() {
@@ -543,6 +544,13 @@ func (x *TopicPartitionStats) GetSubscriberCount() int32 {
return 0
}
+func (x *TopicPartitionStats) GetFollower() string {
+ if x != nil {
+ return x.Follower
+ }
+ return ""
+}
+
type PublisherToPubBalancerRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1049,9 +1057,9 @@ type BrokerPartitionAssignment struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
- Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"`
- LeaderBroker string `protobuf:"bytes,2,opt,name=leader_broker,json=leaderBroker,proto3" json:"leader_broker,omitempty"`
- FollowerBrokers []string `protobuf:"bytes,3,rep,name=follower_brokers,json=followerBrokers,proto3" json:"follower_brokers,omitempty"`
+ Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"`
+ LeaderBroker string `protobuf:"bytes,2,opt,name=leader_broker,json=leaderBroker,proto3" json:"leader_broker,omitempty"`
+ FollowerBroker string `protobuf:"bytes,3,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"`
}
func (x *BrokerPartitionAssignment) Reset() {
@@ -1100,11 +1108,11 @@ func (x *BrokerPartitionAssignment) GetLeaderBroker() string {
return ""
}
-func (x *BrokerPartitionAssignment) GetFollowerBrokers() []string {
+func (x *BrokerPartitionAssignment) GetFollowerBroker() string {
if x != nil {
- return x.FollowerBrokers
+ return x.FollowerBroker
}
- return nil
+ return ""
}
type AssignTopicPartitionsRequest struct {
@@ -1224,7 +1232,8 @@ type SubscriberToSubCoordinatorRequest struct {
// Types that are assignable to Message:
//
// *SubscriberToSubCoordinatorRequest_Init
- // *SubscriberToSubCoordinatorRequest_Ack
+ // *SubscriberToSubCoordinatorRequest_AckAssignment
+ // *SubscriberToSubCoordinatorRequest_AckUnAssignment
Message isSubscriberToSubCoordinatorRequest_Message `protobuf_oneof:"message"`
}
@@ -1274,9 +1283,16 @@ func (x *SubscriberToSubCoordinatorRequest) GetInit() *SubscriberToSubCoordinato
return nil
}
-func (x *SubscriberToSubCoordinatorRequest) GetAck() *SubscriberToSubCoordinatorRequest_AckMessage {
- if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorRequest_Ack); ok {
- return x.Ack
+func (x *SubscriberToSubCoordinatorRequest) GetAckAssignment() *SubscriberToSubCoordinatorRequest_AckAssignmentMessage {
+ if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorRequest_AckAssignment); ok {
+ return x.AckAssignment
+ }
+ return nil
+}
+
+func (x *SubscriberToSubCoordinatorRequest) GetAckUnAssignment() *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage {
+ if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorRequest_AckUnAssignment); ok {
+ return x.AckUnAssignment
}
return nil
}
@@ -1289,13 +1305,21 @@ type SubscriberToSubCoordinatorRequest_Init struct {
Init *SubscriberToSubCoordinatorRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"`
}
-type SubscriberToSubCoordinatorRequest_Ack struct {
- Ack *SubscriberToSubCoordinatorRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"`
+type SubscriberToSubCoordinatorRequest_AckAssignment struct {
+ AckAssignment *SubscriberToSubCoordinatorRequest_AckAssignmentMessage `protobuf:"bytes,2,opt,name=ack_assignment,json=ackAssignment,proto3,oneof"`
+}
+
+type SubscriberToSubCoordinatorRequest_AckUnAssignment struct {
+ AckUnAssignment *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage `protobuf:"bytes,3,opt,name=ack_un_assignment,json=ackUnAssignment,proto3,oneof"`
}
func (*SubscriberToSubCoordinatorRequest_Init) isSubscriberToSubCoordinatorRequest_Message() {}
-func (*SubscriberToSubCoordinatorRequest_Ack) isSubscriberToSubCoordinatorRequest_Message() {}
+func (*SubscriberToSubCoordinatorRequest_AckAssignment) isSubscriberToSubCoordinatorRequest_Message() {
+}
+
+func (*SubscriberToSubCoordinatorRequest_AckUnAssignment) isSubscriberToSubCoordinatorRequest_Message() {
+}
type SubscriberToSubCoordinatorResponse struct {
state protoimpl.MessageState
@@ -1305,6 +1329,7 @@ type SubscriberToSubCoordinatorResponse struct {
// Types that are assignable to Message:
//
// *SubscriberToSubCoordinatorResponse_Assignment_
+ // *SubscriberToSubCoordinatorResponse_UnAssignment_
Message isSubscriberToSubCoordinatorResponse_Message `protobuf_oneof:"message"`
}
@@ -1354,6 +1379,13 @@ func (x *SubscriberToSubCoordinatorResponse) GetAssignment() *SubscriberToSubCoo
return nil
}
+func (x *SubscriberToSubCoordinatorResponse) GetUnAssignment() *SubscriberToSubCoordinatorResponse_UnAssignment {
+ if x, ok := x.GetMessage().(*SubscriberToSubCoordinatorResponse_UnAssignment_); ok {
+ return x.UnAssignment
+ }
+ return nil
+}
+
type isSubscriberToSubCoordinatorResponse_Message interface {
isSubscriberToSubCoordinatorResponse_Message()
}
@@ -1362,16 +1394,24 @@ type SubscriberToSubCoordinatorResponse_Assignment_ struct {
Assignment *SubscriberToSubCoordinatorResponse_Assignment `protobuf:"bytes,1,opt,name=assignment,proto3,oneof"`
}
+type SubscriberToSubCoordinatorResponse_UnAssignment_ struct {
+ UnAssignment *SubscriberToSubCoordinatorResponse_UnAssignment `protobuf:"bytes,2,opt,name=un_assignment,json=unAssignment,proto3,oneof"`
+}
+
func (*SubscriberToSubCoordinatorResponse_Assignment_) isSubscriberToSubCoordinatorResponse_Message() {
}
+func (*SubscriberToSubCoordinatorResponse_UnAssignment_) isSubscriberToSubCoordinatorResponse_Message() {
+}
+
// ////////////////////////////////////////////////
type ControlMessage struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
- IsClose bool `protobuf:"varint,1,opt,name=is_close,json=isClose,proto3" json:"is_close,omitempty"`
+ IsClose bool `protobuf:"varint,1,opt,name=is_close,json=isClose,proto3" json:"is_close,omitempty"`
+ PublisherName string `protobuf:"bytes,2,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"`
}
func (x *ControlMessage) Reset() {
@@ -1413,6 +1453,13 @@ func (x *ControlMessage) GetIsClose() bool {
return false
}
+func (x *ControlMessage) GetPublisherName() string {
+ if x != nil {
+ return x.PublisherName
+ }
+ return ""
+}
+
type DataMessage struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1493,8 +1540,7 @@ type PublishMessageRequest struct {
//
// *PublishMessageRequest_Init
// *PublishMessageRequest_Data
- Message isPublishMessageRequest_Message `protobuf_oneof:"message"`
- Sequence int64 `protobuf:"varint,3,opt,name=sequence,proto3" json:"sequence,omitempty"`
+ Message isPublishMessageRequest_Message `protobuf_oneof:"message"`
}
func (x *PublishMessageRequest) Reset() {
@@ -1550,13 +1596,6 @@ func (x *PublishMessageRequest) GetData() *DataMessage {
return nil
}
-func (x *PublishMessageRequest) GetSequence() int64 {
- if x != nil {
- return x.Sequence
- }
- return 0
-}
-
type isPublishMessageRequest_Message interface {
isPublishMessageRequest_Message()
}
@@ -1954,6 +1993,148 @@ func (*SubscribeMessageResponse_Ctrl) isSubscribeMessageResponse_Message() {}
func (*SubscribeMessageResponse_Data) isSubscribeMessageResponse_Message() {}
+type SubscribeFollowMeRequest struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ // Types that are assignable to Message:
+ //
+ // *SubscribeFollowMeRequest_Init
+ // *SubscribeFollowMeRequest_Ack
+ // *SubscribeFollowMeRequest_Close
+ Message isSubscribeFollowMeRequest_Message `protobuf_oneof:"message"`
+}
+
+func (x *SubscribeFollowMeRequest) Reset() {
+ *x = SubscribeFollowMeRequest{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[31]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscribeFollowMeRequest) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscribeFollowMeRequest) ProtoMessage() {}
+
+func (x *SubscribeFollowMeRequest) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_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 SubscribeFollowMeRequest.ProtoReflect.Descriptor instead.
+func (*SubscribeFollowMeRequest) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{31}
+}
+
+func (m *SubscribeFollowMeRequest) GetMessage() isSubscribeFollowMeRequest_Message {
+ if m != nil {
+ return m.Message
+ }
+ return nil
+}
+
+func (x *SubscribeFollowMeRequest) GetInit() *SubscribeFollowMeRequest_InitMessage {
+ if x, ok := x.GetMessage().(*SubscribeFollowMeRequest_Init); ok {
+ return x.Init
+ }
+ return nil
+}
+
+func (x *SubscribeFollowMeRequest) GetAck() *SubscribeFollowMeRequest_AckMessage {
+ if x, ok := x.GetMessage().(*SubscribeFollowMeRequest_Ack); ok {
+ return x.Ack
+ }
+ return nil
+}
+
+func (x *SubscribeFollowMeRequest) GetClose() *SubscribeFollowMeRequest_CloseMessage {
+ if x, ok := x.GetMessage().(*SubscribeFollowMeRequest_Close); ok {
+ return x.Close
+ }
+ return nil
+}
+
+type isSubscribeFollowMeRequest_Message interface {
+ isSubscribeFollowMeRequest_Message()
+}
+
+type SubscribeFollowMeRequest_Init struct {
+ Init *SubscribeFollowMeRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"`
+}
+
+type SubscribeFollowMeRequest_Ack struct {
+ Ack *SubscribeFollowMeRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"`
+}
+
+type SubscribeFollowMeRequest_Close struct {
+ Close *SubscribeFollowMeRequest_CloseMessage `protobuf:"bytes,3,opt,name=close,proto3,oneof"`
+}
+
+func (*SubscribeFollowMeRequest_Init) isSubscribeFollowMeRequest_Message() {}
+
+func (*SubscribeFollowMeRequest_Ack) isSubscribeFollowMeRequest_Message() {}
+
+func (*SubscribeFollowMeRequest_Close) isSubscribeFollowMeRequest_Message() {}
+
+type SubscribeFollowMeResponse struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ AckTsNs int64 `protobuf:"varint,1,opt,name=ack_ts_ns,json=ackTsNs,proto3" json:"ack_ts_ns,omitempty"`
+}
+
+func (x *SubscribeFollowMeResponse) Reset() {
+ *x = SubscribeFollowMeResponse{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[32]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscribeFollowMeResponse) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscribeFollowMeResponse) ProtoMessage() {}
+
+func (x *SubscribeFollowMeResponse) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_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 SubscribeFollowMeResponse.ProtoReflect.Descriptor instead.
+func (*SubscribeFollowMeResponse) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{32}
+}
+
+func (x *SubscribeFollowMeResponse) GetAckTsNs() int64 {
+ if x != nil {
+ return x.AckTsNs
+ }
+ return 0
+}
+
type ClosePublishersRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1966,7 +2147,7 @@ type ClosePublishersRequest struct {
func (x *ClosePublishersRequest) Reset() {
*x = ClosePublishersRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[31]
+ mi := &file_mq_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1979,7 +2160,7 @@ func (x *ClosePublishersRequest) String() string {
func (*ClosePublishersRequest) ProtoMessage() {}
func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[31]
+ mi := &file_mq_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1992,7 +2173,7 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead.
func (*ClosePublishersRequest) Descriptor() ([]byte, []int) {
- return file_mq_proto_rawDescGZIP(), []int{31}
+ return file_mq_proto_rawDescGZIP(), []int{33}
}
func (x *ClosePublishersRequest) GetTopic() *Topic {
@@ -2018,7 +2199,7 @@ type ClosePublishersResponse struct {
func (x *ClosePublishersResponse) Reset() {
*x = ClosePublishersResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[32]
+ mi := &file_mq_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2031,7 +2212,7 @@ func (x *ClosePublishersResponse) String() string {
func (*ClosePublishersResponse) ProtoMessage() {}
func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[32]
+ mi := &file_mq_proto_msgTypes[34]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2044,7 +2225,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead.
func (*ClosePublishersResponse) Descriptor() ([]byte, []int) {
- return file_mq_proto_rawDescGZIP(), []int{32}
+ return file_mq_proto_rawDescGZIP(), []int{34}
}
type CloseSubscribersRequest struct {
@@ -2059,7 +2240,7 @@ type CloseSubscribersRequest struct {
func (x *CloseSubscribersRequest) Reset() {
*x = CloseSubscribersRequest{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[33]
+ mi := &file_mq_proto_msgTypes[35]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2072,7 +2253,7 @@ func (x *CloseSubscribersRequest) String() string {
func (*CloseSubscribersRequest) ProtoMessage() {}
func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[33]
+ mi := &file_mq_proto_msgTypes[35]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2085,7 +2266,7 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead.
func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) {
- return file_mq_proto_rawDescGZIP(), []int{33}
+ return file_mq_proto_rawDescGZIP(), []int{35}
}
func (x *CloseSubscribersRequest) GetTopic() *Topic {
@@ -2111,7 +2292,7 @@ type CloseSubscribersResponse struct {
func (x *CloseSubscribersResponse) Reset() {
*x = CloseSubscribersResponse{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[34]
+ mi := &file_mq_proto_msgTypes[36]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2124,7 +2305,7 @@ func (x *CloseSubscribersResponse) String() string {
func (*CloseSubscribersResponse) ProtoMessage() {}
func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[34]
+ mi := &file_mq_proto_msgTypes[36]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2137,7 +2318,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead.
func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) {
- return file_mq_proto_rawDescGZIP(), []int{34}
+ return file_mq_proto_rawDescGZIP(), []int{36}
}
type PublisherToPubBalancerRequest_InitMessage struct {
@@ -2151,7 +2332,7 @@ type PublisherToPubBalancerRequest_InitMessage struct {
func (x *PublisherToPubBalancerRequest_InitMessage) Reset() {
*x = PublisherToPubBalancerRequest_InitMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[36]
+ mi := &file_mq_proto_msgTypes[38]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2164,7 +2345,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string {
func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {}
func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[36]
+ mi := &file_mq_proto_msgTypes[38]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2195,12 +2376,22 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct {
ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"`
ConsumerGroupInstanceId string `protobuf:"bytes,2,opt,name=consumer_group_instance_id,json=consumerGroupInstanceId,proto3" json:"consumer_group_instance_id,omitempty"`
Topic *Topic `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"`
+ // The consumer group instance will be assigned at most max_partition_count partitions.
+ // If the number of partitions is less than the sum of max_partition_count,
+ // the consumer group instance may be assigned partitions less than max_partition_count.
+ // Default is 1.
+ MaxPartitionCount int32 `protobuf:"varint,4,opt,name=max_partition_count,json=maxPartitionCount,proto3" json:"max_partition_count,omitempty"`
+ // If consumer group instance changes, wait for rebalance_seconds before reassigning partitions
+ // Exception: if adding a new consumer group instance and sum of max_partition_count equals the number of partitions,
+ // the rebalance will happen immediately.
+ // Default is 10 seconds.
+ RebalanceSeconds int32 `protobuf:"varint,5,opt,name=rebalance_seconds,json=rebalanceSeconds,proto3" json:"rebalance_seconds,omitempty"`
}
func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() {
*x = SubscriberToSubCoordinatorRequest_InitMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[37]
+ mi := &file_mq_proto_msgTypes[39]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2213,7 +2404,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string {
func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[37]
+ mi := &file_mq_proto_msgTypes[39]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2250,32 +2441,45 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetTopic() *Topic {
return nil
}
-type SubscriberToSubCoordinatorRequest_AckMessage struct {
+func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetMaxPartitionCount() int32 {
+ if x != nil {
+ return x.MaxPartitionCount
+ }
+ return 0
+}
+
+func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetRebalanceSeconds() int32 {
+ if x != nil {
+ return x.RebalanceSeconds
+ }
+ return 0
+}
+
+type SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"`
- TsNs int64 `protobuf:"varint,2,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"`
}
-func (x *SubscriberToSubCoordinatorRequest_AckMessage) Reset() {
- *x = SubscriberToSubCoordinatorRequest_AckMessage{}
+func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Reset() {
+ *x = SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[38]
+ mi := &file_mq_proto_msgTypes[40]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
-func (x *SubscriberToSubCoordinatorRequest_AckMessage) String() string {
+func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) String() string {
return protoimpl.X.MessageStringOf(x)
}
-func (*SubscriberToSubCoordinatorRequest_AckMessage) ProtoMessage() {}
+func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoMessage() {}
-func (x *SubscriberToSubCoordinatorRequest_AckMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[38]
+func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_proto_msgTypes[40]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2286,23 +2490,63 @@ func (x *SubscriberToSubCoordinatorRequest_AckMessage) ProtoReflect() protorefle
return mi.MessageOf(x)
}
-// Deprecated: Use SubscriberToSubCoordinatorRequest_AckMessage.ProtoReflect.Descriptor instead.
-func (*SubscriberToSubCoordinatorRequest_AckMessage) Descriptor() ([]byte, []int) {
+// Deprecated: Use SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage.ProtoReflect.Descriptor instead.
+func (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) Descriptor() ([]byte, []int) {
return file_mq_proto_rawDescGZIP(), []int{21, 1}
}
-func (x *SubscriberToSubCoordinatorRequest_AckMessage) GetPartition() *Partition {
+func (x *SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage) GetPartition() *Partition {
if x != nil {
return x.Partition
}
return nil
}
-func (x *SubscriberToSubCoordinatorRequest_AckMessage) GetTsNs() int64 {
+type SubscriberToSubCoordinatorRequest_AckAssignmentMessage struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"`
+}
+
+func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Reset() {
+ *x = SubscriberToSubCoordinatorRequest_AckAssignmentMessage{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[41]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoMessage() {}
+
+func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_proto_msgTypes[41]
+ 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 SubscriberToSubCoordinatorRequest_AckAssignmentMessage.ProtoReflect.Descriptor instead.
+func (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{21, 2}
+}
+
+func (x *SubscriberToSubCoordinatorRequest_AckAssignmentMessage) GetPartition() *Partition {
if x != nil {
- return x.TsNs
+ return x.Partition
}
- return 0
+ return nil
}
type SubscriberToSubCoordinatorResponse_Assignment struct {
@@ -2310,14 +2554,13 @@ type SubscriberToSubCoordinatorResponse_Assignment struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
- Generation int64 `protobuf:"varint,1,opt,name=generation,proto3" json:"generation,omitempty"`
- PartitionAssignments []*BrokerPartitionAssignment `protobuf:"bytes,2,rep,name=partition_assignments,json=partitionAssignments,proto3" json:"partition_assignments,omitempty"`
+ PartitionAssignment *BrokerPartitionAssignment `protobuf:"bytes,1,opt,name=partition_assignment,json=partitionAssignment,proto3" json:"partition_assignment,omitempty"`
}
func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() {
*x = SubscriberToSubCoordinatorResponse_Assignment{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[39]
+ mi := &file_mq_proto_msgTypes[42]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2330,7 +2573,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string {
func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {}
func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[39]
+ mi := &file_mq_proto_msgTypes[42]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2346,16 +2589,56 @@ func (*SubscriberToSubCoordinatorResponse_Assignment) Descriptor() ([]byte, []in
return file_mq_proto_rawDescGZIP(), []int{22, 0}
}
-func (x *SubscriberToSubCoordinatorResponse_Assignment) GetGeneration() int64 {
+func (x *SubscriberToSubCoordinatorResponse_Assignment) GetPartitionAssignment() *BrokerPartitionAssignment {
if x != nil {
- return x.Generation
+ return x.PartitionAssignment
}
- return 0
+ return nil
}
-func (x *SubscriberToSubCoordinatorResponse_Assignment) GetPartitionAssignments() []*BrokerPartitionAssignment {
+type SubscriberToSubCoordinatorResponse_UnAssignment struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"`
+}
+
+func (x *SubscriberToSubCoordinatorResponse_UnAssignment) Reset() {
+ *x = SubscriberToSubCoordinatorResponse_UnAssignment{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[43]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscriberToSubCoordinatorResponse_UnAssignment) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscriberToSubCoordinatorResponse_UnAssignment) ProtoMessage() {}
+
+func (x *SubscriberToSubCoordinatorResponse_UnAssignment) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_proto_msgTypes[43]
+ 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 SubscriberToSubCoordinatorResponse_UnAssignment.ProtoReflect.Descriptor instead.
+func (*SubscriberToSubCoordinatorResponse_UnAssignment) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{22, 1}
+}
+
+func (x *SubscriberToSubCoordinatorResponse_UnAssignment) GetPartition() *Partition {
if x != nil {
- return x.PartitionAssignments
+ return x.Partition
}
return nil
}
@@ -2365,17 +2648,17 @@ type PublishMessageRequest_InitMessage struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
- Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
- Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"`
- AckInterval int32 `protobuf:"varint,3,opt,name=ack_interval,json=ackInterval,proto3" json:"ack_interval,omitempty"`
- FollowerBrokers []string `protobuf:"bytes,4,rep,name=follower_brokers,json=followerBrokers,proto3" json:"follower_brokers,omitempty"`
- PublisherName string `protobuf:"bytes,5,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"` // for debugging
+ Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+ Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"`
+ AckInterval int32 `protobuf:"varint,3,opt,name=ack_interval,json=ackInterval,proto3" json:"ack_interval,omitempty"`
+ FollowerBroker string `protobuf:"bytes,4,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"`
+ PublisherName string `protobuf:"bytes,5,opt,name=publisher_name,json=publisherName,proto3" json:"publisher_name,omitempty"` // for debugging
}
func (x *PublishMessageRequest_InitMessage) Reset() {
*x = PublishMessageRequest_InitMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[40]
+ mi := &file_mq_proto_msgTypes[44]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2388,7 +2671,7 @@ func (x *PublishMessageRequest_InitMessage) String() string {
func (*PublishMessageRequest_InitMessage) ProtoMessage() {}
func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[40]
+ mi := &file_mq_proto_msgTypes[44]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2425,11 +2708,11 @@ func (x *PublishMessageRequest_InitMessage) GetAckInterval() int32 {
return 0
}
-func (x *PublishMessageRequest_InitMessage) GetFollowerBrokers() []string {
+func (x *PublishMessageRequest_InitMessage) GetFollowerBroker() string {
if x != nil {
- return x.FollowerBrokers
+ return x.FollowerBroker
}
- return nil
+ return ""
}
func (x *PublishMessageRequest_InitMessage) GetPublisherName() string {
@@ -2451,7 +2734,7 @@ type PublishFollowMeRequest_InitMessage struct {
func (x *PublishFollowMeRequest_InitMessage) Reset() {
*x = PublishFollowMeRequest_InitMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[41]
+ mi := &file_mq_proto_msgTypes[45]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2464,7 +2747,7 @@ func (x *PublishFollowMeRequest_InitMessage) String() string {
func (*PublishFollowMeRequest_InitMessage) ProtoMessage() {}
func (x *PublishFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[41]
+ mi := &file_mq_proto_msgTypes[45]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2505,7 +2788,7 @@ type PublishFollowMeRequest_FlushMessage struct {
func (x *PublishFollowMeRequest_FlushMessage) Reset() {
*x = PublishFollowMeRequest_FlushMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[42]
+ mi := &file_mq_proto_msgTypes[46]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2518,7 +2801,7 @@ func (x *PublishFollowMeRequest_FlushMessage) String() string {
func (*PublishFollowMeRequest_FlushMessage) ProtoMessage() {}
func (x *PublishFollowMeRequest_FlushMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[42]
+ mi := &file_mq_proto_msgTypes[46]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2550,7 +2833,7 @@ type PublishFollowMeRequest_CloseMessage struct {
func (x *PublishFollowMeRequest_CloseMessage) Reset() {
*x = PublishFollowMeRequest_CloseMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[43]
+ mi := &file_mq_proto_msgTypes[47]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2563,7 +2846,7 @@ func (x *PublishFollowMeRequest_CloseMessage) String() string {
func (*PublishFollowMeRequest_CloseMessage) ProtoMessage() {}
func (x *PublishFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[43]
+ mi := &file_mq_proto_msgTypes[47]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2590,13 +2873,14 @@ type SubscribeMessageRequest_InitMessage struct {
Topic *Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"`
PartitionOffset *PartitionOffset `protobuf:"bytes,5,opt,name=partition_offset,json=partitionOffset,proto3" json:"partition_offset,omitempty"`
Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"`
- FollowerBrokers []string `protobuf:"bytes,7,rep,name=follower_brokers,json=followerBrokers,proto3" json:"follower_brokers,omitempty"`
+ FollowerBroker string `protobuf:"bytes,7,opt,name=follower_broker,json=followerBroker,proto3" json:"follower_broker,omitempty"`
+ Concurrency int32 `protobuf:"varint,8,opt,name=concurrency,proto3" json:"concurrency,omitempty"`
}
func (x *SubscribeMessageRequest_InitMessage) Reset() {
*x = SubscribeMessageRequest_InitMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[44]
+ mi := &file_mq_proto_msgTypes[48]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2609,7 +2893,7 @@ func (x *SubscribeMessageRequest_InitMessage) String() string {
func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {}
func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[44]
+ mi := &file_mq_proto_msgTypes[48]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2667,11 +2951,18 @@ func (x *SubscribeMessageRequest_InitMessage) GetFilter() string {
return ""
}
-func (x *SubscribeMessageRequest_InitMessage) GetFollowerBrokers() []string {
+func (x *SubscribeMessageRequest_InitMessage) GetFollowerBroker() string {
if x != nil {
- return x.FollowerBrokers
+ return x.FollowerBroker
}
- return nil
+ return ""
+}
+
+func (x *SubscribeMessageRequest_InitMessage) GetConcurrency() int32 {
+ if x != nil {
+ return x.Concurrency
+ }
+ return 0
}
type SubscribeMessageRequest_AckMessage struct {
@@ -2679,13 +2970,14 @@ type SubscribeMessageRequest_AckMessage struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
- Sequence int64 `protobuf:"varint,1,opt,name=sequence,proto3" json:"sequence,omitempty"`
+ Sequence int64 `protobuf:"varint,1,opt,name=sequence,proto3" json:"sequence,omitempty"`
+ Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
}
func (x *SubscribeMessageRequest_AckMessage) Reset() {
*x = SubscribeMessageRequest_AckMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[45]
+ mi := &file_mq_proto_msgTypes[49]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2698,7 +2990,7 @@ func (x *SubscribeMessageRequest_AckMessage) String() string {
func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {}
func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[45]
+ mi := &file_mq_proto_msgTypes[49]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2721,6 +3013,13 @@ func (x *SubscribeMessageRequest_AckMessage) GetSequence() int64 {
return 0
}
+func (x *SubscribeMessageRequest_AckMessage) GetKey() []byte {
+ if x != nil {
+ return x.Key
+ }
+ return nil
+}
+
type SubscribeMessageResponse_SubscribeCtrlMessage struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -2734,7 +3033,7 @@ type SubscribeMessageResponse_SubscribeCtrlMessage struct {
func (x *SubscribeMessageResponse_SubscribeCtrlMessage) Reset() {
*x = SubscribeMessageResponse_SubscribeCtrlMessage{}
if protoimpl.UnsafeEnabled {
- mi := &file_mq_proto_msgTypes[46]
+ mi := &file_mq_proto_msgTypes[50]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -2747,7 +3046,7 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) String() string {
func (*SubscribeMessageResponse_SubscribeCtrlMessage) ProtoMessage() {}
func (x *SubscribeMessageResponse_SubscribeCtrlMessage) ProtoReflect() protoreflect.Message {
- mi := &file_mq_proto_msgTypes[46]
+ mi := &file_mq_proto_msgTypes[50]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -2784,6 +3083,154 @@ func (x *SubscribeMessageResponse_SubscribeCtrlMessage) GetIsEndOfTopic() bool {
return false
}
+type SubscribeFollowMeRequest_InitMessage struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+ Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"`
+ ConsumerGroup string `protobuf:"bytes,3,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"`
+}
+
+func (x *SubscribeFollowMeRequest_InitMessage) Reset() {
+ *x = SubscribeFollowMeRequest_InitMessage{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[51]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscribeFollowMeRequest_InitMessage) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscribeFollowMeRequest_InitMessage) ProtoMessage() {}
+
+func (x *SubscribeFollowMeRequest_InitMessage) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_proto_msgTypes[51]
+ 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 SubscribeFollowMeRequest_InitMessage.ProtoReflect.Descriptor instead.
+func (*SubscribeFollowMeRequest_InitMessage) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{31, 0}
+}
+
+func (x *SubscribeFollowMeRequest_InitMessage) GetTopic() *Topic {
+ if x != nil {
+ return x.Topic
+ }
+ return nil
+}
+
+func (x *SubscribeFollowMeRequest_InitMessage) GetPartition() *Partition {
+ if x != nil {
+ return x.Partition
+ }
+ return nil
+}
+
+func (x *SubscribeFollowMeRequest_InitMessage) GetConsumerGroup() string {
+ if x != nil {
+ return x.ConsumerGroup
+ }
+ return ""
+}
+
+type SubscribeFollowMeRequest_AckMessage struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ TsNs int64 `protobuf:"varint,1,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"`
+}
+
+func (x *SubscribeFollowMeRequest_AckMessage) Reset() {
+ *x = SubscribeFollowMeRequest_AckMessage{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[52]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscribeFollowMeRequest_AckMessage) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscribeFollowMeRequest_AckMessage) ProtoMessage() {}
+
+func (x *SubscribeFollowMeRequest_AckMessage) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_proto_msgTypes[52]
+ 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 SubscribeFollowMeRequest_AckMessage.ProtoReflect.Descriptor instead.
+func (*SubscribeFollowMeRequest_AckMessage) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{31, 1}
+}
+
+func (x *SubscribeFollowMeRequest_AckMessage) GetTsNs() int64 {
+ if x != nil {
+ return x.TsNs
+ }
+ return 0
+}
+
+type SubscribeFollowMeRequest_CloseMessage struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+}
+
+func (x *SubscribeFollowMeRequest_CloseMessage) Reset() {
+ *x = SubscribeFollowMeRequest_CloseMessage{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_mq_proto_msgTypes[53]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *SubscribeFollowMeRequest_CloseMessage) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscribeFollowMeRequest_CloseMessage) ProtoMessage() {}
+
+func (x *SubscribeFollowMeRequest_CloseMessage) ProtoReflect() protoreflect.Message {
+ mi := &file_mq_proto_msgTypes[53]
+ 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 SubscribeFollowMeRequest_CloseMessage.ProtoReflect.Descriptor instead.
+func (*SubscribeFollowMeRequest_CloseMessage) Descriptor() ([]byte, []int) {
+ return file_mq_proto_rawDescGZIP(), []int{31, 2}
+}
+
var File_mq_proto protoreflect.FileDescriptor
var file_mq_proto_rawDesc = []byte{
@@ -2843,7 +3290,7 @@ var file_mq_proto_rawDesc = []byte{
0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69,
0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69,
0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
- 0x3a, 0x02, 0x38, 0x01, 0x22, 0xcb, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61,
+ 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe7, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61,
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x05,
0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65,
0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63,
@@ -2856,81 +3303,61 @@ var file_mq_proto_rawDesc = []byte{
0x65, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x75, 0x62, 0x73, 0x63,
0x72, 0x69, 0x62, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
0x05, 0x52, 0x0f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x43, 0x6f, 0x75,
- 0x6e, 0x74, 0x22, 0xd3, 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72,
- 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62,
- 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e,
- 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69,
- 0x6e, 0x69, 0x74, 0x12, 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x48, 0x00, 0x52,
- 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x42, 0x09, 0x0a,
- 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x20, 0x0a, 0x1e, 0x50, 0x75, 0x62, 0x6c,
- 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63,
- 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x42, 0x61,
- 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
- 0x73, 0x74, 0x22, 0x17, 0x0a, 0x15, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70,
- 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa3, 0x01, 0x0a, 0x15,
- 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
- 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63,
- 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f,
- 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69,
- 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63,
- 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15,
- 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72,
- 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70,
- 0x65, 0x22, 0xbb, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54,
- 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c,
- 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
- 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03,
- 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
- 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f,
- 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69,
- 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72,
- 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73,
- 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54,
- 0x79, 0x70, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x22,
- 0x13, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69,
- 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x74, 0x6f,
- 0x70, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73,
- 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52,
- 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x22, 0x46, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75,
- 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
- 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x22,
- 0xb2, 0x01, 0x0a, 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42,
- 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29,
- 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e,
- 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70,
- 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f,
- 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73,
- 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32,
- 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42,
- 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73,
- 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72,
- 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d,
- 0x65, 0x6e, 0x74, 0x73, 0x22, 0xa2, 0x01, 0x0a, 0x19, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50,
+ 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x18, 0x05,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x22, 0xd3,
+ 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75,
+ 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37,
+ 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75,
+ 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61,
+ 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74,
+ 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12,
+ 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19,
+ 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72,
+ 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x48, 0x00, 0x52, 0x05, 0x73, 0x74, 0x61,
+ 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x65, 0x22, 0x20, 0x0a, 0x1e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65,
+ 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63,
+ 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x17,
+ 0x0a, 0x15, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa3, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6e, 0x66,
+ 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
+ 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x27, 0x0a, 0x0f,
+ 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
+ 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f,
+ 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68,
+ 0x65, 0x6d, 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70,
+ 0x65, 0x52, 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x22, 0xbb, 0x01,
+ 0x0a, 0x16, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b,
+ 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73,
+ 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27,
+ 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72,
+ 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73,
+ 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65,
- 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
- 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09,
- 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61,
- 0x64, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0c, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x29,
- 0x0a, 0x10, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65,
- 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77,
- 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x22, 0xf2, 0x01, 0x0a, 0x1c, 0x41, 0x73,
- 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
- 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f,
+ 0x6e, 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x74, 0x79,
+ 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d,
+ 0x61, 0x5f, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52,
+ 0x0a, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x54, 0x79, 0x70, 0x65, 0x22, 0x13, 0x0a, 0x11, 0x4c,
+ 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73,
+ 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69,
+ 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x06, 0x74, 0x6f, 0x70,
+ 0x69, 0x63, 0x73, 0x22, 0x46, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70,
+ 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54,
+ 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x22, 0xb2, 0x01, 0x0a, 0x1a,
+ 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65,
+ 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f,
0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05,
0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f,
@@ -2940,301 +3367,393 @@ var file_mq_proto_rawDesc = []byte{
0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74,
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73,
- 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a,
- 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01,
- 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x22, 0x1f,
- 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72,
- 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0xca, 0x03, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f,
- 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
- 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53,
- 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
- 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x4e, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18,
- 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
- 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54,
- 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0x9c, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69,
- 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73,
- 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12,
- 0x3b, 0x0a, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75,
- 0x70, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f,
- 0x75, 0x70, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05,
- 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63,
- 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x1a, 0x58, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
- 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
- 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05,
- 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e,
- 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9b, 0x02, 0x0a,
- 0x22, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62,
- 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e,
- 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
- 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f,
- 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
- 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65,
- 0x6e, 0x74, 0x1a, 0x8a, 0x01, 0x0a, 0x0a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e,
- 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f,
- 0x6e, 0x12, 0x5c, 0x0a, 0x15, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61,
- 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b,
- 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
- 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41,
- 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x14, 0x70, 0x61, 0x72, 0x74, 0x69,
- 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42,
- 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x2b, 0x0a, 0x0e, 0x43, 0x6f,
- 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x08,
- 0x69, 0x73, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07,
- 0x69, 0x73, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x7c, 0x0a, 0x0b, 0x44, 0x61, 0x74, 0x61, 0x4d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
- 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x13,
- 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74,
- 0x73, 0x4e, 0x73, 0x12, 0x30, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28,
- 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
- 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52,
- 0x04, 0x63, 0x74, 0x72, 0x6c, 0x22, 0x9d, 0x03, 0x0a, 0x15, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73,
- 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
- 0x45, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e,
- 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62,
- 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
- 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00,
- 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
- 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
- 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65,
- 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65,
- 0x6e, 0x63, 0x65, 0x1a, 0xe4, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35,
- 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x22, 0xa0, 0x01, 0x0a, 0x19, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69,
+ 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35,
+ 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74,
- 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x74,
- 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x61, 0x63, 0x6b,
- 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f, 0x6c, 0x6c,
- 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03,
- 0x28, 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b,
- 0x65, 0x72, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72,
- 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x75, 0x62,
- 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x74, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68,
- 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
- 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e,
- 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x68, 0x6f, 0x75,
- 0x6c, 0x64, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b,
- 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0xd8, 0x03, 0x0a, 0x16,
- 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
- 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f,
- 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f,
- 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61,
- 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12,
- 0x49, 0x0a, 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31,
- 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75,
- 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x65, 0x48, 0x00, 0x52, 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x12, 0x49, 0x0a, 0x05, 0x63, 0x6c,
- 0x6f, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68,
- 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e,
- 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05,
- 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x1a, 0x6f, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73,
- 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20,
+ 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f,
+ 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6c, 0x65,
+ 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x6f,
+ 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f,
+ 0x6b, 0x65, 0x72, 0x22, 0xf2, 0x01, 0x0a, 0x1c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f,
+ 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12,
- 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72,
- 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x23, 0x0a, 0x0c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x1a, 0x0e, 0x0a, 0x0c, 0x43,
- 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x35, 0x0a, 0x17, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73,
- 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0x8a, 0x04,
- 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x47, 0x0a, 0x04, 0x69, 0x6e, 0x69,
- 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
- 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49,
+ 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
+ 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18,
+ 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
+ 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69,
+ 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a,
+ 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41,
+ 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73,
+ 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69,
+ 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72,
+ 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73,
+ 0x44, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x22, 0x1f, 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69,
+ 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x82, 0x06, 0x0a, 0x21, 0x53, 0x75,
+ 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f,
+ 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
+ 0x51, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e,
+ 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62,
+ 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72,
+ 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49,
0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e,
- 0x69, 0x74, 0x12, 0x44, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
- 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53,
- 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52,
- 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0xaa, 0x02, 0x0a, 0x0b, 0x49, 0x6e, 0x69,
- 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73,
- 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12,
- 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64,
- 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a,
- 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69,
- 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x48, 0x0a, 0x10, 0x70, 0x61, 0x72, 0x74,
- 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x05, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65,
- 0x74, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73,
- 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f,
- 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x07,
- 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72,
- 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x1a, 0x28, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x42,
- 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa7, 0x02, 0x0a, 0x18, 0x53,
- 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x51, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
- 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x75,
- 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61,
- 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x7c, 0x0a, 0x14, 0x53,
- 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73,
- 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x27, 0x0a, 0x10, 0x69, 0x73, 0x5f,
- 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x02, 0x20,
- 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x53, 0x74, 0x72, 0x65,
- 0x61, 0x6d, 0x12, 0x25, 0x0a, 0x0f, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f,
- 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x45,
- 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73,
- 0x73, 0x61, 0x67, 0x65, 0x22, 0x65, 0x0a, 0x16, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62,
- 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29,
- 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e,
- 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70,
- 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69,
- 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52,
- 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x19, 0x0a, 0x17, 0x43,
- 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x66, 0x0a, 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53,
- 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
- 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c,
- 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x1a,
- 0x0a, 0x18, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
- 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x4c, 0x0a, 0x18, 0x50, 0x61,
- 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61,
- 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45,
- 0x53, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54,
- 0x5f, 0x49, 0x4e, 0x5f, 0x4d, 0x45, 0x4d, 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06,
- 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x10, 0x02, 0x32, 0xde, 0x0a, 0x0a, 0x10, 0x53, 0x65, 0x61,
- 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x63, 0x0a,
- 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65,
- 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
- 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65,
- 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b,
- 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x00, 0x12, 0x79, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54,
- 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x12, 0x2b, 0x2e, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c,
- 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63,
- 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65, 0x73, 0x73,
+ 0x69, 0x74, 0x12, 0x6d, 0x0a, 0x0e, 0x61, 0x63, 0x6b, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e,
+ 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72,
+ 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e,
+ 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x41,
+ 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+ 0x48, 0x00, 0x52, 0x0d, 0x61, 0x63, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e,
+ 0x74, 0x12, 0x74, 0x0a, 0x11, 0x61, 0x63, 0x6b, 0x5f, 0x75, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69,
+ 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73,
+ 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64,
+ 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63,
+ 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x61, 0x63, 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73,
+ 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0xf9, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74,
+ 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75,
+ 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x3b,
+ 0x0a, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70,
+ 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75,
+ 0x70, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74,
+ 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52,
+ 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2e, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61,
+ 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20,
+ 0x01, 0x28, 0x05, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
+ 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2b, 0x0a, 0x11, 0x72, 0x65, 0x62, 0x61, 0x6c, 0x61,
+ 0x6e, 0x63, 0x65, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28,
+ 0x05, 0x52, 0x10, 0x72, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x65, 0x63, 0x6f,
+ 0x6e, 0x64, 0x73, 0x1a, 0x4f, 0x0a, 0x16, 0x41, 0x63, 0x6b, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69,
+ 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x35, 0x0a,
+ 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
+ 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69,
+ 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4d, 0x0a, 0x14, 0x41, 0x63, 0x6b, 0x41, 0x73, 0x73, 0x69, 0x67,
+ 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x35, 0x0a, 0x09,
+ 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50,
+ 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
+ 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa5,
+ 0x03, 0x0a, 0x22, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53,
+ 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d,
+ 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73,
+ 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
+ 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61,
+ 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, 0x69,
+ 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e,
+ 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x0d, 0x75, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67,
+ 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6d, 0x65,
+ 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63,
+ 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69,
+ 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x55, 0x6e,
+ 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x75, 0x6e,
+ 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x68, 0x0a, 0x0a, 0x41, 0x73,
+ 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x5a, 0x0a, 0x14, 0x70, 0x61, 0x72, 0x74,
+ 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69,
+ 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74,
+ 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52,
+ 0x13, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
+ 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x45, 0x0a, 0x0c, 0x55, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
+ 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
+ 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x6d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x52, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f,
+ 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x63,
+ 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x43, 0x6c,
+ 0x6f, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72,
+ 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x75, 0x62,
+ 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x7c, 0x0a, 0x0b, 0x44, 0x61,
+ 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76,
+ 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
+ 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,
+ 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x12, 0x30, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x04,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
+ 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x65, 0x52, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x22, 0xff, 0x02, 0x0a, 0x15, 0x50, 0x75, 0x62,
+ 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x12, 0x45, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
+ 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74,
+ 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0xe2, 0x01, 0x0a, 0x0b, 0x49,
+ 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f,
+ 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73,
+ 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05,
+ 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
+ 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
+ 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c,
+ 0x61, 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x05, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12,
+ 0x27, 0x0a, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b,
+ 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77,
+ 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x75, 0x62, 0x6c,
+ 0x69, 0x73, 0x68, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x0d, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x42,
+ 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x74, 0x0a, 0x16, 0x50, 0x75,
+ 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70,
+ 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75,
+ 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53,
+ 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72,
+ 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a,
+ 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65,
+ 0x22, 0xd8, 0x03, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c,
+ 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x04, 0x69,
+ 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68,
- 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x5a, 0x0a,
- 0x0d, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x22,
- 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61,
- 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
- 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
- 0x62, 0x2e, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52,
- 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x4c, 0x69, 0x73,
- 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63,
- 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69,
- 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0e,
- 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x23,
- 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f,
- 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
- 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69,
- 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x12, 0x4c,
- 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72,
- 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
- 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b,
- 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x65, 0x73,
- 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70,
- 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x15, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
- 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12,
- 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41,
- 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74,
- 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x65,
- 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67,
- 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73,
- 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x43, 0x6c,
- 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12, 0x24, 0x2e,
- 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f,
- 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
- 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65,
- 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x10,
- 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73,
- 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
- 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73,
- 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72,
- 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72,
- 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
- 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43,
- 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
- 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62,
- 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x61, 0x0a, 0x0e, 0x50, 0x75, 0x62,
- 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e, 0x6d, 0x65,
+ 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e,
+ 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69,
+ 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
+ 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04,
+ 0x64, 0x61, 0x74, 0x61, 0x12, 0x49, 0x0a, 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
+ 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77,
+ 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x12,
+ 0x49, 0x0a, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31,
+ 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75,
+ 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x1a, 0x6f, 0x0a, 0x0b, 0x49, 0x6e,
+ 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70,
+ 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74,
+ 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
+ 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x23, 0x0a, 0x0c, 0x46,
+ 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74,
+ 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73,
+ 0x1a, 0x0e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+ 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x35, 0x0a, 0x17, 0x50,
+ 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x73,
+ 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61, 0x63, 0x6b, 0x54, 0x73,
+ 0x4e, 0x73, 0x22, 0xbc, 0x04, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
+ 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x47,
+ 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73,
+ 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
+ 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x44, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
+ 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0xca, 0x02,
+ 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a,
+ 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47,
+ 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72,
+ 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75,
+ 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f,
+ 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74,
+ 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
+ 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x48, 0x0a,
+ 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65,
+ 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
+ 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65,
+ 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12,
+ 0x27, 0x0a, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b,
+ 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77,
+ 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63,
+ 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63,
+ 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x1a, 0x3a, 0x0a, 0x0a, 0x41, 0x63,
+ 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75,
+ 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75,
+ 0x65, 0x6e, 0x63, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+ 0x65, 0x22, 0xa7, 0x02, 0x0a, 0x18, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x51,
+ 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73,
+ 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70,
+ 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x74,
+ 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, 0x72,
+ 0x6c, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44,
+ 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61,
+ 0x74, 0x61, 0x1a, 0x7c, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43,
+ 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72,
+ 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72,
+ 0x12, 0x27, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x74,
+ 0x72, 0x65, 0x61, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x73, 0x45, 0x6e,
+ 0x64, 0x4f, 0x66, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x25, 0x0a, 0x0f, 0x69, 0x73, 0x5f,
+ 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01,
+ 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63,
+ 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xcf, 0x03, 0x0a, 0x18,
+ 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d,
+ 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x48, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69,
+ 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46,
+ 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49,
+ 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e,
+ 0x69, 0x74, 0x12, 0x45, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53,
+ 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65,
+ 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x12, 0x4b, 0x0a, 0x05, 0x63, 0x6c, 0x6f,
+ 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
+ 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52,
+ 0x05, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x1a, 0x96, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
+ 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69,
+ 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
+ 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70,
+ 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73,
+ 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x1a,
+ 0x21, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x13, 0x0a,
+ 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73,
+ 0x4e, 0x73, 0x1a, 0x0e, 0x0a, 0x0c, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x37, 0x0a,
+ 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77,
+ 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x09, 0x61, 0x63,
+ 0x6b, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x61,
+ 0x63, 0x6b, 0x54, 0x73, 0x4e, 0x73, 0x22, 0x65, 0x0a, 0x16, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50,
+ 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54,
+ 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75,
+ 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x19, 0x0a,
+ 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x66, 0x0a, 0x17, 0x43, 0x6c, 0x6f, 0x73,
+ 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01,
+ 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
+ 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20,
+ 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73,
+ 0x22, 0x1a, 0x0a, 0x18, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
+ 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x4c, 0x0a, 0x18,
+ 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x53,
+ 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52, 0x4c,
+ 0x49, 0x45, 0x53, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45,
+ 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x4d, 0x45, 0x4d, 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12, 0x0a,
+ 0x0a, 0x06, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x10, 0x02, 0x32, 0xca, 0x0b, 0x0a, 0x10, 0x53,
+ 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12,
+ 0x63, 0x0a, 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61,
+ 0x64, 0x65, 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
+ 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61,
+ 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72,
+ 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65,
+ 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x12, 0x2b,
+ 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75,
+ 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61,
+ 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65,
0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69,
- 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
- 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x10,
- 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
- 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
- 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
- 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
- 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
- 0x00, 0x30, 0x01, 0x12, 0x64, 0x0a, 0x0f, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f,
- 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69,
- 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c,
- 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d,
- 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c,
- 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x42, 0x4f, 0x0a, 0x0c, 0x73, 0x65, 0x61,
- 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, 0x6d, 0x71, 0x42, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61,
- 0x67, 0x65, 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 0x67, 0x69,
- 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64,
- 0x66, 0x73, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65,
- 0x64, 0x2f, 0x70, 0x62, 0x2f, 0x6d, 0x71, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74,
- 0x6f, 0x33,
+ 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65,
+ 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12,
+ 0x5a, 0x0a, 0x0d, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73,
+ 0x12, 0x22, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
+ 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67,
+ 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63,
+ 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x4c,
+ 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x2e, 0x6d, 0x65, 0x73, 0x73,
+ 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70,
+ 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f,
+ 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d,
+ 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63,
+ 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
+ 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
+ 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f,
+ 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a,
+ 0x12, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b,
+ 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
+ 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72,
+ 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d,
+ 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b,
+ 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65,
+ 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x15, 0x41, 0x73, 0x73, 0x69,
+ 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62,
+ 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74,
+ 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e,
+ 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73,
+ 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
+ 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f,
+ 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x12,
+ 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43,
+ 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e,
+ 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73,
+ 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63,
+ 0x0a, 0x10, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
+ 0x72, 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
+ 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
+ 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73,
+ 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75,
+ 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x1a, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
+ 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74,
+ 0x6f, 0x72, 0x12, 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
+ 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75,
+ 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
+ 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53,
+ 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73,
+ 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x61, 0x0a, 0x0e, 0x50,
+ 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e,
+ 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62,
+ 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
+ 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+ 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x67,
+ 0x0a, 0x10, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x65, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
+ 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61,
+ 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73,
+ 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
+ 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+ 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x64, 0x0a, 0x0f, 0x50, 0x75, 0x62, 0x6c, 0x69,
+ 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73,
+ 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e,
+ 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x68, 0x0a,
+ 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77,
+ 0x4d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70,
+ 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f,
+ 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x65, 0x73,
+ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72,
+ 0x69, 0x62, 0x65, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
+ 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x42, 0x4f, 0x0a, 0x0c, 0x73, 0x65, 0x61, 0x77, 0x65,
+ 0x65, 0x64, 0x66, 0x73, 0x2e, 0x6d, 0x71, 0x42, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+ 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 0x67, 0x69, 0x74, 0x68,
+ 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73,
+ 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f,
+ 0x70, 0x62, 0x2f, 0x6d, 0x71, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -3250,72 +3769,79 @@ func file_mq_proto_rawDescGZIP() []byte {
}
var file_mq_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
-var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 47)
+var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 54)
var file_mq_proto_goTypes = []interface{}{
- (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType
- (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest
- (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse
- (*Topic)(nil), // 3: messaging_pb.Topic
- (*Partition)(nil), // 4: messaging_pb.Partition
- (*Offset)(nil), // 5: messaging_pb.Offset
- (*PartitionOffset)(nil), // 6: messaging_pb.PartitionOffset
- (*BrokerStats)(nil), // 7: messaging_pb.BrokerStats
- (*TopicPartitionStats)(nil), // 8: messaging_pb.TopicPartitionStats
- (*PublisherToPubBalancerRequest)(nil), // 9: messaging_pb.PublisherToPubBalancerRequest
- (*PublisherToPubBalancerResponse)(nil), // 10: messaging_pb.PublisherToPubBalancerResponse
- (*BalanceTopicsRequest)(nil), // 11: messaging_pb.BalanceTopicsRequest
- (*BalanceTopicsResponse)(nil), // 12: messaging_pb.BalanceTopicsResponse
- (*ConfigureTopicRequest)(nil), // 13: messaging_pb.ConfigureTopicRequest
- (*ConfigureTopicResponse)(nil), // 14: messaging_pb.ConfigureTopicResponse
- (*ListTopicsRequest)(nil), // 15: messaging_pb.ListTopicsRequest
- (*ListTopicsResponse)(nil), // 16: messaging_pb.ListTopicsResponse
- (*LookupTopicBrokersRequest)(nil), // 17: messaging_pb.LookupTopicBrokersRequest
- (*LookupTopicBrokersResponse)(nil), // 18: messaging_pb.LookupTopicBrokersResponse
- (*BrokerPartitionAssignment)(nil), // 19: messaging_pb.BrokerPartitionAssignment
- (*AssignTopicPartitionsRequest)(nil), // 20: messaging_pb.AssignTopicPartitionsRequest
- (*AssignTopicPartitionsResponse)(nil), // 21: messaging_pb.AssignTopicPartitionsResponse
- (*SubscriberToSubCoordinatorRequest)(nil), // 22: messaging_pb.SubscriberToSubCoordinatorRequest
- (*SubscriberToSubCoordinatorResponse)(nil), // 23: messaging_pb.SubscriberToSubCoordinatorResponse
- (*ControlMessage)(nil), // 24: messaging_pb.ControlMessage
- (*DataMessage)(nil), // 25: messaging_pb.DataMessage
- (*PublishMessageRequest)(nil), // 26: messaging_pb.PublishMessageRequest
- (*PublishMessageResponse)(nil), // 27: messaging_pb.PublishMessageResponse
- (*PublishFollowMeRequest)(nil), // 28: messaging_pb.PublishFollowMeRequest
- (*PublishFollowMeResponse)(nil), // 29: messaging_pb.PublishFollowMeResponse
- (*SubscribeMessageRequest)(nil), // 30: messaging_pb.SubscribeMessageRequest
- (*SubscribeMessageResponse)(nil), // 31: messaging_pb.SubscribeMessageResponse
- (*ClosePublishersRequest)(nil), // 32: messaging_pb.ClosePublishersRequest
- (*ClosePublishersResponse)(nil), // 33: messaging_pb.ClosePublishersResponse
- (*CloseSubscribersRequest)(nil), // 34: messaging_pb.CloseSubscribersRequest
- (*CloseSubscribersResponse)(nil), // 35: messaging_pb.CloseSubscribersResponse
- nil, // 36: messaging_pb.BrokerStats.StatsEntry
- (*PublisherToPubBalancerRequest_InitMessage)(nil), // 37: messaging_pb.PublisherToPubBalancerRequest.InitMessage
- (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 38: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
- (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage
- (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 40: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
- (*PublishMessageRequest_InitMessage)(nil), // 41: messaging_pb.PublishMessageRequest.InitMessage
- (*PublishFollowMeRequest_InitMessage)(nil), // 42: messaging_pb.PublishFollowMeRequest.InitMessage
- (*PublishFollowMeRequest_FlushMessage)(nil), // 43: messaging_pb.PublishFollowMeRequest.FlushMessage
- (*PublishFollowMeRequest_CloseMessage)(nil), // 44: messaging_pb.PublishFollowMeRequest.CloseMessage
- (*SubscribeMessageRequest_InitMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.InitMessage
- (*SubscribeMessageRequest_AckMessage)(nil), // 46: messaging_pb.SubscribeMessageRequest.AckMessage
- (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 47: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
- (*schema_pb.RecordType)(nil), // 48: schema_pb.RecordType
+ (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType
+ (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest
+ (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse
+ (*Topic)(nil), // 3: messaging_pb.Topic
+ (*Partition)(nil), // 4: messaging_pb.Partition
+ (*Offset)(nil), // 5: messaging_pb.Offset
+ (*PartitionOffset)(nil), // 6: messaging_pb.PartitionOffset
+ (*BrokerStats)(nil), // 7: messaging_pb.BrokerStats
+ (*TopicPartitionStats)(nil), // 8: messaging_pb.TopicPartitionStats
+ (*PublisherToPubBalancerRequest)(nil), // 9: messaging_pb.PublisherToPubBalancerRequest
+ (*PublisherToPubBalancerResponse)(nil), // 10: messaging_pb.PublisherToPubBalancerResponse
+ (*BalanceTopicsRequest)(nil), // 11: messaging_pb.BalanceTopicsRequest
+ (*BalanceTopicsResponse)(nil), // 12: messaging_pb.BalanceTopicsResponse
+ (*ConfigureTopicRequest)(nil), // 13: messaging_pb.ConfigureTopicRequest
+ (*ConfigureTopicResponse)(nil), // 14: messaging_pb.ConfigureTopicResponse
+ (*ListTopicsRequest)(nil), // 15: messaging_pb.ListTopicsRequest
+ (*ListTopicsResponse)(nil), // 16: messaging_pb.ListTopicsResponse
+ (*LookupTopicBrokersRequest)(nil), // 17: messaging_pb.LookupTopicBrokersRequest
+ (*LookupTopicBrokersResponse)(nil), // 18: messaging_pb.LookupTopicBrokersResponse
+ (*BrokerPartitionAssignment)(nil), // 19: messaging_pb.BrokerPartitionAssignment
+ (*AssignTopicPartitionsRequest)(nil), // 20: messaging_pb.AssignTopicPartitionsRequest
+ (*AssignTopicPartitionsResponse)(nil), // 21: messaging_pb.AssignTopicPartitionsResponse
+ (*SubscriberToSubCoordinatorRequest)(nil), // 22: messaging_pb.SubscriberToSubCoordinatorRequest
+ (*SubscriberToSubCoordinatorResponse)(nil), // 23: messaging_pb.SubscriberToSubCoordinatorResponse
+ (*ControlMessage)(nil), // 24: messaging_pb.ControlMessage
+ (*DataMessage)(nil), // 25: messaging_pb.DataMessage
+ (*PublishMessageRequest)(nil), // 26: messaging_pb.PublishMessageRequest
+ (*PublishMessageResponse)(nil), // 27: messaging_pb.PublishMessageResponse
+ (*PublishFollowMeRequest)(nil), // 28: messaging_pb.PublishFollowMeRequest
+ (*PublishFollowMeResponse)(nil), // 29: messaging_pb.PublishFollowMeResponse
+ (*SubscribeMessageRequest)(nil), // 30: messaging_pb.SubscribeMessageRequest
+ (*SubscribeMessageResponse)(nil), // 31: messaging_pb.SubscribeMessageResponse
+ (*SubscribeFollowMeRequest)(nil), // 32: messaging_pb.SubscribeFollowMeRequest
+ (*SubscribeFollowMeResponse)(nil), // 33: messaging_pb.SubscribeFollowMeResponse
+ (*ClosePublishersRequest)(nil), // 34: messaging_pb.ClosePublishersRequest
+ (*ClosePublishersResponse)(nil), // 35: messaging_pb.ClosePublishersResponse
+ (*CloseSubscribersRequest)(nil), // 36: messaging_pb.CloseSubscribersRequest
+ (*CloseSubscribersResponse)(nil), // 37: messaging_pb.CloseSubscribersResponse
+ nil, // 38: messaging_pb.BrokerStats.StatsEntry
+ (*PublisherToPubBalancerRequest_InitMessage)(nil), // 39: messaging_pb.PublisherToPubBalancerRequest.InitMessage
+ (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 40: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
+ (*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage)(nil), // 41: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage
+ (*SubscriberToSubCoordinatorRequest_AckAssignmentMessage)(nil), // 42: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage
+ (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 43: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
+ (*SubscriberToSubCoordinatorResponse_UnAssignment)(nil), // 44: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment
+ (*PublishMessageRequest_InitMessage)(nil), // 45: messaging_pb.PublishMessageRequest.InitMessage
+ (*PublishFollowMeRequest_InitMessage)(nil), // 46: messaging_pb.PublishFollowMeRequest.InitMessage
+ (*PublishFollowMeRequest_FlushMessage)(nil), // 47: messaging_pb.PublishFollowMeRequest.FlushMessage
+ (*PublishFollowMeRequest_CloseMessage)(nil), // 48: messaging_pb.PublishFollowMeRequest.CloseMessage
+ (*SubscribeMessageRequest_InitMessage)(nil), // 49: messaging_pb.SubscribeMessageRequest.InitMessage
+ (*SubscribeMessageRequest_AckMessage)(nil), // 50: messaging_pb.SubscribeMessageRequest.AckMessage
+ (*SubscribeMessageResponse_SubscribeCtrlMessage)(nil), // 51: messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
+ (*SubscribeFollowMeRequest_InitMessage)(nil), // 52: messaging_pb.SubscribeFollowMeRequest.InitMessage
+ (*SubscribeFollowMeRequest_AckMessage)(nil), // 53: messaging_pb.SubscribeFollowMeRequest.AckMessage
+ (*SubscribeFollowMeRequest_CloseMessage)(nil), // 54: messaging_pb.SubscribeFollowMeRequest.CloseMessage
+ (*schema_pb.RecordType)(nil), // 55: schema_pb.RecordType
}
var file_mq_proto_depIdxs = []int32{
3, // 0: messaging_pb.Offset.topic:type_name -> messaging_pb.Topic
6, // 1: messaging_pb.Offset.partition_offsets:type_name -> messaging_pb.PartitionOffset
4, // 2: messaging_pb.PartitionOffset.partition:type_name -> messaging_pb.Partition
0, // 3: messaging_pb.PartitionOffset.start_type:type_name -> messaging_pb.PartitionOffsetStartType
- 36, // 4: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry
+ 38, // 4: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry
3, // 5: messaging_pb.TopicPartitionStats.topic:type_name -> messaging_pb.Topic
4, // 6: messaging_pb.TopicPartitionStats.partition:type_name -> messaging_pb.Partition
- 37, // 7: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage
+ 39, // 7: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage
7, // 8: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats
3, // 9: messaging_pb.ConfigureTopicRequest.topic:type_name -> messaging_pb.Topic
- 48, // 10: messaging_pb.ConfigureTopicRequest.record_type:type_name -> schema_pb.RecordType
+ 55, // 10: messaging_pb.ConfigureTopicRequest.record_type:type_name -> schema_pb.RecordType
19, // 11: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
- 48, // 12: messaging_pb.ConfigureTopicResponse.record_type:type_name -> schema_pb.RecordType
+ 55, // 12: messaging_pb.ConfigureTopicResponse.record_type:type_name -> schema_pb.RecordType
3, // 13: messaging_pb.ListTopicsResponse.topics:type_name -> messaging_pb.Topic
3, // 14: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> messaging_pb.Topic
3, // 15: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> messaging_pb.Topic
@@ -3323,63 +3849,74 @@ var file_mq_proto_depIdxs = []int32{
4, // 17: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition
3, // 18: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic
19, // 19: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
- 38, // 20: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
- 39, // 21: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage
- 40, // 22: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
- 24, // 23: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage
- 41, // 24: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage
- 25, // 25: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage
- 42, // 26: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage
- 25, // 27: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage
- 43, // 28: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage
- 44, // 29: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage
- 45, // 30: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage
- 46, // 31: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage
- 47, // 32: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
- 25, // 33: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage
- 3, // 34: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic
- 3, // 35: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic
- 8, // 36: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats
- 3, // 37: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic
- 4, // 38: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition
- 19, // 39: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment
- 3, // 40: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic
- 4, // 41: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition
- 3, // 42: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic
- 4, // 43: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition
- 3, // 44: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic
- 6, // 45: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset
- 1, // 46: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest
- 9, // 47: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest
- 11, // 48: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest
- 15, // 49: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest
- 13, // 50: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest
- 17, // 51: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest
- 20, // 52: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest
- 32, // 53: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest
- 34, // 54: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest
- 22, // 55: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest
- 26, // 56: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest
- 30, // 57: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest
- 28, // 58: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest
- 2, // 59: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse
- 10, // 60: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse
- 12, // 61: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse
- 16, // 62: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse
- 14, // 63: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse
- 18, // 64: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse
- 21, // 65: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse
- 33, // 66: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse
- 35, // 67: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse
- 23, // 68: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse
- 27, // 69: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse
- 31, // 70: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse
- 29, // 71: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse
- 59, // [59:72] is the sub-list for method output_type
- 46, // [46:59] is the sub-list for method input_type
- 46, // [46:46] is the sub-list for extension type_name
- 46, // [46:46] is the sub-list for extension extendee
- 0, // [0:46] is the sub-list for field type_name
+ 40, // 20: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage
+ 42, // 21: messaging_pb.SubscriberToSubCoordinatorRequest.ack_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage
+ 41, // 22: messaging_pb.SubscriberToSubCoordinatorRequest.ack_un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage
+ 43, // 23: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment
+ 44, // 24: messaging_pb.SubscriberToSubCoordinatorResponse.un_assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment
+ 24, // 25: messaging_pb.DataMessage.ctrl:type_name -> messaging_pb.ControlMessage
+ 45, // 26: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage
+ 25, // 27: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage
+ 46, // 28: messaging_pb.PublishFollowMeRequest.init:type_name -> messaging_pb.PublishFollowMeRequest.InitMessage
+ 25, // 29: messaging_pb.PublishFollowMeRequest.data:type_name -> messaging_pb.DataMessage
+ 47, // 30: messaging_pb.PublishFollowMeRequest.flush:type_name -> messaging_pb.PublishFollowMeRequest.FlushMessage
+ 48, // 31: messaging_pb.PublishFollowMeRequest.close:type_name -> messaging_pb.PublishFollowMeRequest.CloseMessage
+ 49, // 32: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage
+ 50, // 33: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage
+ 51, // 34: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.SubscribeCtrlMessage
+ 25, // 35: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage
+ 52, // 36: messaging_pb.SubscribeFollowMeRequest.init:type_name -> messaging_pb.SubscribeFollowMeRequest.InitMessage
+ 53, // 37: messaging_pb.SubscribeFollowMeRequest.ack:type_name -> messaging_pb.SubscribeFollowMeRequest.AckMessage
+ 54, // 38: messaging_pb.SubscribeFollowMeRequest.close:type_name -> messaging_pb.SubscribeFollowMeRequest.CloseMessage
+ 3, // 39: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic
+ 3, // 40: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic
+ 8, // 41: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats
+ 3, // 42: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic
+ 4, // 43: messaging_pb.SubscriberToSubCoordinatorRequest.AckUnAssignmentMessage.partition:type_name -> messaging_pb.Partition
+ 4, // 44: messaging_pb.SubscriberToSubCoordinatorRequest.AckAssignmentMessage.partition:type_name -> messaging_pb.Partition
+ 19, // 45: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.partition_assignment:type_name -> messaging_pb.BrokerPartitionAssignment
+ 4, // 46: messaging_pb.SubscriberToSubCoordinatorResponse.UnAssignment.partition:type_name -> messaging_pb.Partition
+ 3, // 47: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic
+ 4, // 48: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition
+ 3, // 49: messaging_pb.PublishFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic
+ 4, // 50: messaging_pb.PublishFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition
+ 3, // 51: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic
+ 6, // 52: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset
+ 3, // 53: messaging_pb.SubscribeFollowMeRequest.InitMessage.topic:type_name -> messaging_pb.Topic
+ 4, // 54: messaging_pb.SubscribeFollowMeRequest.InitMessage.partition:type_name -> messaging_pb.Partition
+ 1, // 55: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest
+ 9, // 56: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest
+ 11, // 57: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest
+ 15, // 58: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest
+ 13, // 59: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest
+ 17, // 60: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest
+ 20, // 61: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest
+ 34, // 62: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest
+ 36, // 63: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest
+ 22, // 64: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest
+ 26, // 65: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest
+ 30, // 66: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest
+ 28, // 67: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest
+ 32, // 68: messaging_pb.SeaweedMessaging.SubscribeFollowMe:input_type -> messaging_pb.SubscribeFollowMeRequest
+ 2, // 69: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse
+ 10, // 70: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse
+ 12, // 71: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse
+ 16, // 72: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse
+ 14, // 73: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse
+ 18, // 74: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse
+ 21, // 75: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse
+ 35, // 76: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse
+ 37, // 77: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse
+ 23, // 78: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse
+ 27, // 79: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse
+ 31, // 80: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse
+ 29, // 81: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse
+ 33, // 82: messaging_pb.SeaweedMessaging.SubscribeFollowMe:output_type -> messaging_pb.SubscribeFollowMeResponse
+ 69, // [69:83] is the sub-list for method output_type
+ 55, // [55:69] is the sub-list for method input_type
+ 55, // [55:55] is the sub-list for extension type_name
+ 55, // [55:55] is the sub-list for extension extendee
+ 0, // [0:55] is the sub-list for field type_name
}
func init() { file_mq_proto_init() }
@@ -3761,7 +4298,7 @@ func file_mq_proto_init() {
}
}
file_mq_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ClosePublishersRequest); i {
+ switch v := v.(*SubscribeFollowMeRequest); i {
case 0:
return &v.state
case 1:
@@ -3773,7 +4310,7 @@ func file_mq_proto_init() {
}
}
file_mq_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ClosePublishersResponse); i {
+ switch v := v.(*SubscribeFollowMeResponse); i {
case 0:
return &v.state
case 1:
@@ -3785,7 +4322,7 @@ func file_mq_proto_init() {
}
}
file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*CloseSubscribersRequest); i {
+ switch v := v.(*ClosePublishersRequest); i {
case 0:
return &v.state
case 1:
@@ -3797,7 +4334,19 @@ func file_mq_proto_init() {
}
}
file_mq_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*CloseSubscribersResponse); i {
+ switch v := v.(*ClosePublishersResponse); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*CloseSubscribersRequest); i {
case 0:
return &v.state
case 1:
@@ -3809,6 +4358,18 @@ func file_mq_proto_init() {
}
}
file_mq_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*CloseSubscribersResponse); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i {
case 0:
return &v.state
@@ -3820,7 +4381,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i {
case 0:
return &v.state
@@ -3832,8 +4393,8 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*SubscriberToSubCoordinatorRequest_AckMessage); i {
+ file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage); i {
case 0:
return &v.state
case 1:
@@ -3844,7 +4405,19 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*SubscriberToSubCoordinatorRequest_AckAssignmentMessage); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i {
case 0:
return &v.state
@@ -3856,7 +4429,19 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*SubscriberToSubCoordinatorResponse_UnAssignment); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PublishMessageRequest_InitMessage); i {
case 0:
return &v.state
@@ -3868,7 +4453,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PublishFollowMeRequest_InitMessage); i {
case 0:
return &v.state
@@ -3880,7 +4465,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PublishFollowMeRequest_FlushMessage); i {
case 0:
return &v.state
@@ -3892,7 +4477,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*PublishFollowMeRequest_CloseMessage); i {
case 0:
return &v.state
@@ -3904,7 +4489,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SubscribeMessageRequest_InitMessage); i {
case 0:
return &v.state
@@ -3916,7 +4501,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SubscribeMessageRequest_AckMessage); i {
case 0:
return &v.state
@@ -3928,7 +4513,7 @@ func file_mq_proto_init() {
return nil
}
}
- file_mq_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
+ file_mq_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*SubscribeMessageResponse_SubscribeCtrlMessage); i {
case 0:
return &v.state
@@ -3940,6 +4525,42 @@ func file_mq_proto_init() {
return nil
}
}
+ file_mq_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*SubscribeFollowMeRequest_InitMessage); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_mq_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*SubscribeFollowMeRequest_AckMessage); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_mq_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*SubscribeFollowMeRequest_CloseMessage); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
}
file_mq_proto_msgTypes[8].OneofWrappers = []interface{}{
(*PublisherToPubBalancerRequest_Init)(nil),
@@ -3947,10 +4568,12 @@ func file_mq_proto_init() {
}
file_mq_proto_msgTypes[21].OneofWrappers = []interface{}{
(*SubscriberToSubCoordinatorRequest_Init)(nil),
- (*SubscriberToSubCoordinatorRequest_Ack)(nil),
+ (*SubscriberToSubCoordinatorRequest_AckAssignment)(nil),
+ (*SubscriberToSubCoordinatorRequest_AckUnAssignment)(nil),
}
file_mq_proto_msgTypes[22].OneofWrappers = []interface{}{
(*SubscriberToSubCoordinatorResponse_Assignment_)(nil),
+ (*SubscriberToSubCoordinatorResponse_UnAssignment_)(nil),
}
file_mq_proto_msgTypes[25].OneofWrappers = []interface{}{
(*PublishMessageRequest_Init)(nil),
@@ -3970,13 +4593,18 @@ func file_mq_proto_init() {
(*SubscribeMessageResponse_Ctrl)(nil),
(*SubscribeMessageResponse_Data)(nil),
}
+ file_mq_proto_msgTypes[31].OneofWrappers = []interface{}{
+ (*SubscribeFollowMeRequest_Init)(nil),
+ (*SubscribeFollowMeRequest_Ack)(nil),
+ (*SubscribeFollowMeRequest_Close)(nil),
+ }
type x struct{}
out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_mq_proto_rawDesc,
NumEnums: 1,
- NumMessages: 47,
+ NumMessages: 54,
NumExtensions: 0,
NumServices: 1,
},
diff --git a/weed/pb/mq_pb/mq_grpc.pb.go b/weed/pb/mq_pb/mq_grpc.pb.go
index 0028f341e..2123415a8 100644
--- a/weed/pb/mq_pb/mq_grpc.pb.go
+++ b/weed/pb/mq_pb/mq_grpc.pb.go
@@ -32,6 +32,7 @@ const (
SeaweedMessaging_PublishMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishMessage"
SeaweedMessaging_SubscribeMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscribeMessage"
SeaweedMessaging_PublishFollowMe_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishFollowMe"
+ SeaweedMessaging_SubscribeFollowMe_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscribeFollowMe"
)
// SeaweedMessagingClient is the client API for SeaweedMessaging service.
@@ -55,9 +56,10 @@ type SeaweedMessagingClient interface {
SubscriberToSubCoordinator(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscriberToSubCoordinatorClient, error)
// data plane for each topic partition
PublishMessage(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishMessageClient, error)
- SubscribeMessage(ctx context.Context, in *SubscribeMessageRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error)
+ SubscribeMessage(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error)
// The lead broker asks a follower broker to follow itself
PublishFollowMe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishFollowMeClient, error)
+ SubscribeFollowMe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeFollowMeClient, error)
}
type seaweedMessagingClient struct {
@@ -233,22 +235,17 @@ func (x *seaweedMessagingPublishMessageClient) Recv() (*PublishMessageResponse,
return m, nil
}
-func (c *seaweedMessagingClient) SubscribeMessage(ctx context.Context, in *SubscribeMessageRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error) {
+func (c *seaweedMessagingClient) SubscribeMessage(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[3], SeaweedMessaging_SubscribeMessage_FullMethodName, opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingSubscribeMessageClient{stream}
- if err := x.ClientStream.SendMsg(in); err != nil {
- return nil, err
- }
- if err := x.ClientStream.CloseSend(); err != nil {
- return nil, err
- }
return x, nil
}
type SeaweedMessaging_SubscribeMessageClient interface {
+ Send(*SubscribeMessageRequest) error
Recv() (*SubscribeMessageResponse, error)
grpc.ClientStream
}
@@ -257,6 +254,10 @@ type seaweedMessagingSubscribeMessageClient struct {
grpc.ClientStream
}
+func (x *seaweedMessagingSubscribeMessageClient) Send(m *SubscribeMessageRequest) error {
+ return x.ClientStream.SendMsg(m)
+}
+
func (x *seaweedMessagingSubscribeMessageClient) Recv() (*SubscribeMessageResponse, error) {
m := new(SubscribeMessageResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
@@ -296,6 +297,40 @@ func (x *seaweedMessagingPublishFollowMeClient) Recv() (*PublishFollowMeResponse
return m, nil
}
+func (c *seaweedMessagingClient) SubscribeFollowMe(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeFollowMeClient, error) {
+ stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[5], SeaweedMessaging_SubscribeFollowMe_FullMethodName, opts...)
+ if err != nil {
+ return nil, err
+ }
+ x := &seaweedMessagingSubscribeFollowMeClient{stream}
+ return x, nil
+}
+
+type SeaweedMessaging_SubscribeFollowMeClient interface {
+ Send(*SubscribeFollowMeRequest) error
+ CloseAndRecv() (*SubscribeFollowMeResponse, error)
+ grpc.ClientStream
+}
+
+type seaweedMessagingSubscribeFollowMeClient struct {
+ grpc.ClientStream
+}
+
+func (x *seaweedMessagingSubscribeFollowMeClient) Send(m *SubscribeFollowMeRequest) error {
+ return x.ClientStream.SendMsg(m)
+}
+
+func (x *seaweedMessagingSubscribeFollowMeClient) CloseAndRecv() (*SubscribeFollowMeResponse, error) {
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ m := new(SubscribeFollowMeResponse)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
// SeaweedMessagingServer is the server API for SeaweedMessaging service.
// All implementations must embed UnimplementedSeaweedMessagingServer
// for forward compatibility
@@ -317,9 +352,10 @@ type SeaweedMessagingServer interface {
SubscriberToSubCoordinator(SeaweedMessaging_SubscriberToSubCoordinatorServer) error
// data plane for each topic partition
PublishMessage(SeaweedMessaging_PublishMessageServer) error
- SubscribeMessage(*SubscribeMessageRequest, SeaweedMessaging_SubscribeMessageServer) error
+ SubscribeMessage(SeaweedMessaging_SubscribeMessageServer) error
// The lead broker asks a follower broker to follow itself
PublishFollowMe(SeaweedMessaging_PublishFollowMeServer) error
+ SubscribeFollowMe(SeaweedMessaging_SubscribeFollowMeServer) error
mustEmbedUnimplementedSeaweedMessagingServer()
}
@@ -360,12 +396,15 @@ func (UnimplementedSeaweedMessagingServer) SubscriberToSubCoordinator(SeaweedMes
func (UnimplementedSeaweedMessagingServer) PublishMessage(SeaweedMessaging_PublishMessageServer) error {
return status.Errorf(codes.Unimplemented, "method PublishMessage not implemented")
}
-func (UnimplementedSeaweedMessagingServer) SubscribeMessage(*SubscribeMessageRequest, SeaweedMessaging_SubscribeMessageServer) error {
+func (UnimplementedSeaweedMessagingServer) SubscribeMessage(SeaweedMessaging_SubscribeMessageServer) error {
return status.Errorf(codes.Unimplemented, "method SubscribeMessage not implemented")
}
func (UnimplementedSeaweedMessagingServer) PublishFollowMe(SeaweedMessaging_PublishFollowMeServer) error {
return status.Errorf(codes.Unimplemented, "method PublishFollowMe not implemented")
}
+func (UnimplementedSeaweedMessagingServer) SubscribeFollowMe(SeaweedMessaging_SubscribeFollowMeServer) error {
+ return status.Errorf(codes.Unimplemented, "method SubscribeFollowMe not implemented")
+}
func (UnimplementedSeaweedMessagingServer) mustEmbedUnimplementedSeaweedMessagingServer() {}
// UnsafeSeaweedMessagingServer may be embedded to opt out of forward compatibility for this service.
@@ -602,15 +641,12 @@ func (x *seaweedMessagingPublishMessageServer) Recv() (*PublishMessageRequest, e
}
func _SeaweedMessaging_SubscribeMessage_Handler(srv interface{}, stream grpc.ServerStream) error {
- m := new(SubscribeMessageRequest)
- if err := stream.RecvMsg(m); err != nil {
- return err
- }
- return srv.(SeaweedMessagingServer).SubscribeMessage(m, &seaweedMessagingSubscribeMessageServer{stream})
+ return srv.(SeaweedMessagingServer).SubscribeMessage(&seaweedMessagingSubscribeMessageServer{stream})
}
type SeaweedMessaging_SubscribeMessageServer interface {
Send(*SubscribeMessageResponse) error
+ Recv() (*SubscribeMessageRequest, error)
grpc.ServerStream
}
@@ -622,6 +658,14 @@ func (x *seaweedMessagingSubscribeMessageServer) Send(m *SubscribeMessageRespons
return x.ServerStream.SendMsg(m)
}
+func (x *seaweedMessagingSubscribeMessageServer) Recv() (*SubscribeMessageRequest, error) {
+ m := new(SubscribeMessageRequest)
+ if err := x.ServerStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
func _SeaweedMessaging_PublishFollowMe_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(SeaweedMessagingServer).PublishFollowMe(&seaweedMessagingPublishFollowMeServer{stream})
}
@@ -648,6 +692,32 @@ func (x *seaweedMessagingPublishFollowMeServer) Recv() (*PublishFollowMeRequest,
return m, nil
}
+func _SeaweedMessaging_SubscribeFollowMe_Handler(srv interface{}, stream grpc.ServerStream) error {
+ return srv.(SeaweedMessagingServer).SubscribeFollowMe(&seaweedMessagingSubscribeFollowMeServer{stream})
+}
+
+type SeaweedMessaging_SubscribeFollowMeServer interface {
+ SendAndClose(*SubscribeFollowMeResponse) error
+ Recv() (*SubscribeFollowMeRequest, error)
+ grpc.ServerStream
+}
+
+type seaweedMessagingSubscribeFollowMeServer struct {
+ grpc.ServerStream
+}
+
+func (x *seaweedMessagingSubscribeFollowMeServer) SendAndClose(m *SubscribeFollowMeResponse) error {
+ return x.ServerStream.SendMsg(m)
+}
+
+func (x *seaweedMessagingSubscribeFollowMeServer) Recv() (*SubscribeFollowMeRequest, error) {
+ m := new(SubscribeFollowMeRequest)
+ if err := x.ServerStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
// SeaweedMessaging_ServiceDesc is the grpc.ServiceDesc for SeaweedMessaging service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
@@ -711,6 +781,7 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
StreamName: "SubscribeMessage",
Handler: _SeaweedMessaging_SubscribeMessage_Handler,
ServerStreams: true,
+ ClientStreams: true,
},
{
StreamName: "PublishFollowMe",
@@ -718,6 +789,11 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
ServerStreams: true,
ClientStreams: true,
},
+ {
+ StreamName: "SubscribeFollowMe",
+ Handler: _SeaweedMessaging_SubscribeFollowMe_Handler,
+ ClientStreams: true,
+ },
},
Metadata: "mq.proto",
}