You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by aa...@apache.org on 2022/07/14 07:44:13 UTC

[rocketmq-clients] branch master updated: Carding structure (#45)

This is an automated email from the ASF dual-hosted git repository.

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git


The following commit(s) were added to refs/heads/master by this push:
     new 36291b8  Carding structure (#45)
36291b8 is described below

commit 36291b83c711cfe75fbb8f07c7bd22cd58f1b032
Author: guyinyou <36...@users.noreply.github.com>
AuthorDate: Thu Jul 14 15:44:07 2022 +0800

    Carding structure (#45)
    
    * protocol v2 (#30)
    
    * Carding structure
    
    Co-authored-by: zheyu <sa...@aliyun.com>
---
 golang/broker.go                            |  364 ---
 golang/broker_options.go                    |  112 -
 golang/client.go                            |  232 ++
 golang/client_manager.go                    |  224 ++
 golang/{ns_options.go => client_options.go} |   62 +-
 golang/config.go                            |    6 +-
 golang/conn.go                              |   67 +-
 golang/consumer.go                          |  268 ++-
 golang/example/consumer/main.go             |   76 +-
 golang/example/producer/main.go             |   41 +-
 golang/go.mod                               |    5 +-
 golang/go.sum                               |    4 +
 golang/message.go                           |   21 +
 golang/metadata/metadata.go                 |   13 +-
 golang/ns.go                                |  179 --
 golang/producer.go                          |  217 +-
 golang/producer_options.go                  |   10 +-
 golang/protocol/v1/admin.pb.go              |  246 --
 golang/protocol/v1/definition.pb.go         | 1407 -----------
 golang/protocol/v1/service.pb.go            | 3342 --------------------------
 golang/protocol/v2/admin.pb.go              |  300 +++
 golang/protocol/v2/admin_grpc.pb.go         |  105 +
 golang/protocol/v2/definition.pb.go         | 2635 +++++++++++++++++++++
 golang/protocol/v2/service.pb.go            | 3387 +++++++++++++++++++++++++++
 golang/protocol/v2/service_grpc.pb.go       |  665 ++++++
 golang/publishing_loadBalancer.go           |   86 +
 golang/publishing_message.go                |   95 +
 golang/rpc_client.go                        |  258 ++
 golang/rpc_client_options.go                |   97 +
 golang/utils/utils.go                       |   20 +
 30 files changed, 8595 insertions(+), 5949 deletions(-)

diff --git a/golang/broker.go b/golang/broker.go
deleted file mode 100644
index 5ff75e1..0000000
--- a/golang/broker.go
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package golang
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"sync"
-	"sync/atomic"
-	"time"
-
-	"github.com/apache/rocketmq-clients/golang/metadata"
-	innerOS "github.com/apache/rocketmq-clients/golang/pkg/os"
-	"github.com/apache/rocketmq-clients/golang/pkg/ticker"
-	v1 "github.com/apache/rocketmq-clients/golang/protocol/v1"
-
-	"github.com/google/uuid"
-	"github.com/lithammer/shortuuid/v4"
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/resolver"
-	"google.golang.org/protobuf/types/known/durationpb"
-	"google.golang.org/protobuf/types/known/timestamppb"
-)
-
-var (
-	ErrNoAvailableBrokers = errors.New("rocketmq: no available brokers")
-)
-
-type Broker interface {
-	SetPartition(partitions []*v1.Partition)
-	Send(ctx context.Context, msg *Message) (*SendReceipt, error)
-	QueryAssignment(ctx context.Context, topic string) ([]*v1.Assignment, error)
-	ReceiveMessage(ctx context.Context, partition *v1.Partition, topic string) ([]*MessageExt, error)
-	AckMessage(ctx context.Context, msg *MessageExt) error
-	GracefulStop() error
-}
-
-type BrokerFunc func(config *Config, route []*v1.Partition, opts ...BrokerOption) (Broker, error)
-
-var _ = Broker(&broker{})
-
-type broker struct {
-	opts       brokerOptions
-	cc         resolver.ClientConn
-	partitions atomic.Value
-	mux        sync.Mutex
-	conn       ClientConn
-	msc        v1.MessagingServiceClient
-	done       chan struct{}
-}
-
-func NewBroker(config *Config, route []*v1.Partition, opts ...BrokerOption) (Broker, error) {
-	b := &broker{
-		opts: defaultBrokerOptions,
-	}
-	for _, opt := range opts {
-		opt.apply(&b.opts)
-	}
-	b.SetPartition(route)
-	config.Endpoint = b.URL()
-	connOpts := append(b.opts.connOptions, WithDialOptions(
-		grpc.WithResolvers(b),
-	))
-	conn, err := b.opts.clientConnFunc(config, connOpts...)
-	if err != nil {
-		return nil, err
-	}
-	b.done = make(chan struct{})
-	b.conn = conn
-	b.msc = v1.NewMessagingServiceClient(conn.Conn())
-	return b, nil
-}
-
-func (b *broker) SetPartition(partitions []*v1.Partition) {
-	endpoints := make(map[string]*v1.Broker)
-	for i := 0; i < len(partitions); i++ {
-		if !b.opts.producer {
-			if partitions[i].Permission == v1.Permission_NONE {
-				continue
-			}
-			if partitions[i].Broker.Id != 0 {
-				continue
-			}
-		}
-		for _, ep := range partitions[i].GetBroker().GetEndpoints().GetAddresses() {
-			endpoints[fmt.Sprintf("%s:%d", ep.Host, ep.Port)] = partitions[i].GetBroker()
-		}
-	}
-	b.partitions.Store(endpoints)
-}
-
-func (b *broker) Build(target resolver.Target,
-	cc resolver.ClientConn,
-	opts resolver.BuildOptions) (resolver.Resolver, error) {
-	b.cc = cc
-	b.ResolveNow(resolver.ResolveNowOptions{})
-	return b, nil
-}
-func (b *broker) Close() {}
-
-func (b *broker) doResolve(opts resolver.ResolveNowOptions) {
-	brokers, ok := b.partitions.Load().(map[string]*v1.Broker)
-	if !ok || len(brokers) == 0 {
-		return
-	}
-	var addrs []resolver.Address
-	for addr, broker := range brokers {
-		addrs = append(addrs, resolver.Address{
-			Addr:       addr,
-			ServerName: broker.GetName(),
-		})
-	}
-	b.mux.Lock()
-	defer b.mux.Unlock()
-	b.cc.UpdateState(resolver.State{
-		Addresses: addrs,
-	})
-}
-
-func (b *broker) ResolveNow(opts resolver.ResolveNowOptions) {
-	b.doResolve(opts)
-}
-
-func (b *broker) Scheme() string {
-	return "brokers"
-}
-
-func (b *broker) URL() string {
-	return fmt.Sprintf("%s:///%s-%s", b.Scheme(), metadata.Rocketmq, shortuuid.New())
-}
-
-func (b *broker) GracefulStop() error {
-	close(b.done)
-	return b.conn.Close()
-}
-
-func (b *broker) Send(ctx context.Context, msg *Message) (*SendReceipt, error) {
-	resp, err := b.msc.SendMessage(ctx, b.getSendMessageRequest(msg))
-	if err != nil {
-		return nil, err
-	}
-	return &SendReceipt{
-		MessageID: resp.GetMessageId(),
-	}, nil
-}
-
-func (b *broker) HeartBeat(req *v1.HeartbeatRequest) {
-	f := func() {
-		ctx, _ := context.WithTimeout(context.TODO(), b.opts.timeout)
-		_, _ = b.msc.Heartbeat(ctx, req)
-	}
-	f()
-	ticker.OnceAndTick(f, b.opts.heartbeatDuration, b.done)
-}
-
-func (b *broker) QueryAssignment(ctx context.Context, topic string) ([]*v1.Assignment, error) {
-	b.HeartBeat(b.getHeartBeatRequest(topic))
-	req, err := b.getQueryAssignmentRequest(topic)
-	if err != nil {
-		return nil, err
-	}
-	assignment, err := b.msc.QueryAssignment(ctx, req)
-	if err != nil {
-		return nil, err
-	}
-	if assignment.GetCommon().GetStatus().GetCode() != int32(codes.OK) {
-		return nil, fmt.Errorf("QueryAssignment err = %s", assignment.String())
-	}
-
-	if len(assignment.Assignments) == 0 {
-		return nil, errors.New("rocketmq: no available assignments")
-	}
-	return assignment.Assignments, nil
-}
-
-func (b *broker) ReceiveMessage(ctx context.Context, partition *v1.Partition, topic string) ([]*MessageExt, error) {
-	response, err := b.msc.ReceiveMessage(ctx, b.getReceiveMessageRequest(partition, topic))
-	if err != nil {
-		return nil, err
-	}
-	if response.GetCommon().GetStatus().GetCode() != int32(codes.OK) {
-		return nil, fmt.Errorf("ReceiveMessage err = %s", response.String())
-	}
-	var ret []*MessageExt
-	for _, msg := range response.Messages {
-		ret = append(ret, &MessageExt{
-			MessageID:     msg.GetSystemAttribute().GetMessageId(),
-			ReceiptHandle: msg.GetSystemAttribute().GetReceiptHandle(),
-			Message: Message{
-				Topic:      msg.GetTopic().GetName(),
-				Body:       msg.GetBody(),
-				Tag:        msg.GetSystemAttribute().GetTag(),
-				Keys:       msg.GetSystemAttribute().GetKeys(),
-				Properties: msg.GetUserAttribute(),
-			},
-		})
-	}
-	return ret, nil
-}
-
-func (b *broker) AckMessage(ctx context.Context, msg *MessageExt) error {
-	response, err := b.msc.AckMessage(ctx, b.getAckMessageRequest(msg))
-	if err != nil {
-		return err
-	}
-	if response.GetCommon().GetStatus().GetCode() != int32(codes.OK) {
-		return fmt.Errorf("AckMessage err = %s", response.String())
-	}
-	return nil
-}
-
-func (b *broker) getSendMessageRequest(msg *Message) *v1.SendMessageRequest {
-	return &v1.SendMessageRequest{
-		Message: &v1.Message{
-			Topic: &v1.Resource{
-				ResourceNamespace: b.conn.Config().NameSpace,
-				Name:              msg.Topic,
-			},
-			SystemAttribute: &v1.SystemAttribute{
-				Tag:           msg.Tag,
-				MessageGroup:  b.conn.Config().Group,
-				MessageType:   v1.MessageType_NORMAL,
-				BodyEncoding:  v1.Encoding_IDENTITY,
-				BornHost:      innerOS.Hostname(),
-				MessageId:     uuid.New().String(),
-				BornTimestamp: timestamppb.Now(),
-			},
-			Body: msg.Body,
-		},
-	}
-}
-
-func (b *broker) getQueryAssignmentRequest(topic string) (ret *v1.QueryAssignmentRequest, err error) {
-	brokers, ok := b.partitions.Load().(map[string]*v1.Broker)
-	if !ok || len(brokers) == 0 {
-		err = ErrNoAvailableBrokers
-		return
-	}
-
-	for _, broker := range brokers {
-		ret = &v1.QueryAssignmentRequest{
-			Topic: &v1.Resource{
-				ResourceNamespace: b.conn.Config().NameSpace,
-				Name:              topic,
-			},
-			Group: &v1.Resource{
-				ResourceNamespace: b.conn.Config().NameSpace,
-				Name:              b.conn.Config().Group,
-			},
-			ClientId: b.ID(),
-			Endpoints: &v1.Endpoints{
-				Scheme:    broker.GetEndpoints().GetScheme(),
-				Addresses: broker.GetEndpoints().GetAddresses(),
-			},
-		}
-		break
-	}
-	return
-}
-
-func (b *broker) getHeartBeatRequest(topic string) *v1.HeartbeatRequest {
-	return &v1.HeartbeatRequest{
-		ClientId: b.ID(),
-		FifoFlag: false,
-		ClientData: &v1.HeartbeatRequest_ConsumerData{
-			ConsumerData: &v1.ConsumerData{
-				Group: &v1.Resource{
-					ResourceNamespace: b.conn.Config().NameSpace,
-					Name:              b.conn.Config().Group,
-				},
-				Subscriptions: []*v1.SubscriptionEntry{
-					{
-						Topic: &v1.Resource{
-							ResourceNamespace: b.conn.Config().NameSpace,
-							Name:              topic,
-						},
-						Expression: &v1.FilterExpression{
-							Type:       v1.FilterType_TAG,
-							Expression: "*",
-						},
-					},
-				},
-				ConsumeType:   v1.ConsumeMessageType_PASSIVE,
-				ConsumeModel:  v1.ConsumeModel_CLUSTERING,
-				ConsumePolicy: v1.ConsumePolicy_RESUME,
-				DeadLetterPolicy: &v1.DeadLetterPolicy{
-					MaxDeliveryAttempts: 17,
-				},
-			},
-		},
-	}
-}
-
-func (b *broker) getReceiveMessageRequest(partition *v1.Partition, topic string) *v1.ReceiveMessageRequest {
-	return &v1.ReceiveMessageRequest{
-		Group: &v1.Resource{
-			ResourceNamespace: b.conn.Config().NameSpace,
-			Name:              b.conn.Config().Group,
-		},
-		ClientId: b.ID(),
-		Partition: &v1.Partition{
-			Topic: &v1.Resource{
-				ResourceNamespace: b.conn.Config().NameSpace,
-				Name:              topic,
-			},
-			Id: partition.Id,
-			Broker: &v1.Broker{
-				Name: partition.Broker.Name,
-			},
-		},
-		FilterExpression: &v1.FilterExpression{
-			Type:       v1.FilterType_TAG,
-			Expression: "*",
-		},
-		ConsumePolicy:     v1.ConsumePolicy_RESUME,
-		BatchSize:         32,
-		InvisibleDuration: durationpb.New(time.Millisecond * 15 * 60 * 1000),
-		AwaitTime:         durationpb.New(time.Millisecond * 0),
-		FifoFlag:          false,
-	}
-}
-
-func (b *broker) getAckMessageRequest(msg *MessageExt) *v1.AckMessageRequest {
-	return &v1.AckMessageRequest{
-		Group: &v1.Resource{
-			ResourceNamespace: b.conn.Config().NameSpace,
-			Name:              b.conn.Config().Group,
-		},
-		Topic: &v1.Resource{
-			ResourceNamespace: b.conn.Config().NameSpace,
-			Name:              msg.Topic,
-		},
-		ClientId:  b.ID(),
-		MessageId: msg.MessageID,
-		Handle: &v1.AckMessageRequest_ReceiptHandle{
-			ReceiptHandle: msg.ReceiptHandle,
-		},
-	}
-}
-
-func (b *broker) ID() string {
-	return b.opts.id
-}
-
-func init() {
-	resolver.Register(&broker{})
-}
diff --git a/golang/broker_options.go b/golang/broker_options.go
deleted file mode 100644
index 0563fcf..0000000
--- a/golang/broker_options.go
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package golang
-
-import (
-	"fmt"
-	"math/rand"
-	"time"
-
-	"github.com/apache/rocketmq-clients/golang/metadata"
-	"github.com/lithammer/shortuuid/v4"
-)
-
-type brokerOptions struct {
-	id                  string
-	producer            bool
-	heartbeatDuration   time.Duration
-	healthCheckDuration time.Duration
-	timeout             time.Duration
-	clientConnFunc      ClientConnFunc
-	connOptions         []ConnOption
-}
-
-var defaultBrokerOptions = brokerOptions{
-	id:                  fmt.Sprintf("%s@%d@%s", metadata.Rocketmq, rand.Int(), shortuuid.New()),
-	heartbeatDuration:   time.Second * 10,
-	healthCheckDuration: time.Second * 15,
-	timeout:             time.Second * 5,
-	clientConnFunc:      NewClientConn,
-}
-
-// A BrokerOption sets options such as tls.Config, etc.
-type BrokerOption interface {
-	apply(*brokerOptions)
-}
-
-// funcBrokerOption wraps a function that modifies options into an implementation of
-// the ConnOption interface.
-type funcBrokerOption struct {
-	f func(options *brokerOptions)
-}
-
-func (fpo *funcBrokerOption) apply(po *brokerOptions) {
-	fpo.f(po)
-}
-
-func newFuncOption(f func(options *brokerOptions)) *funcBrokerOption {
-	return &funcBrokerOption{
-		f: f,
-	}
-}
-
-// WithBrokerClientConnFunc returns a BrokerOption that sets ClientConnFunc for Broker.
-// Default is NewClientConn.
-func WithBrokerClientConnFunc(f ClientConnFunc) BrokerOption {
-	return newFuncOption(func(o *brokerOptions) {
-		o.clientConnFunc = f
-	})
-}
-
-// WithBrokerConnOption returns a BrokerOption that sets ConnOption for Broker.
-func WithBrokerConnOption(opts ...ConnOption) BrokerOption {
-	return newFuncOption(func(o *brokerOptions) {
-		o.connOptions = append(o.connOptions, opts...)
-	})
-}
-
-// WithProducer returns a BrokerOption that sets producer flag for Broker.
-func WithProducer() BrokerOption {
-	return newFuncOption(func(o *brokerOptions) {
-		o.producer = true
-	})
-}
-
-// WithHeartbeatDuration returns a BrokerOption that sets heartbeatDuration for Broker.
-// Default is 10s.
-func WithHeartbeatDuration(d time.Duration) BrokerOption {
-	return newFuncOption(func(o *brokerOptions) {
-		o.heartbeatDuration = d
-	})
-}
-
-// WithHealthCheckDuration returns a BrokerOption that sets healthCheckDuration for Broker.
-// Default is 15s.
-func WithHealthCheckDuration(d time.Duration) BrokerOption {
-	return newFuncOption(func(o *brokerOptions) {
-		o.healthCheckDuration = d
-	})
-}
-
-// WithBrokerTimeout returns a BrokerOption that sets time for Broker when heartbeat and health check.
-// Default is 5s.
-func WithBrokerTimeout(d time.Duration) BrokerOption {
-	return newFuncOption(func(o *brokerOptions) {
-		o.timeout = d
-	})
-}
diff --git a/golang/client.go b/golang/client.go
new file mode 100644
index 0000000..5ca623d
--- /dev/null
+++ b/golang/client.go
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package golang
+
+import (
+	"context"
+	"crypto/hmac"
+	"crypto/sha1"
+	"encoding/hex"
+	"errors"
+	"fmt"
+	"log"
+	"math/rand"
+	"net/url"
+	"reflect"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+
+	innerMD "github.com/apache/rocketmq-clients/golang/metadata"
+	"github.com/apache/rocketmq-clients/golang/pkg/ticker"
+	v2 "github.com/apache/rocketmq-clients/golang/protocol/v2"
+	"github.com/google/uuid"
+	"github.com/lithammer/shortuuid/v4"
+	"google.golang.org/grpc/metadata"
+)
+
+type Client interface {
+	GetClientID() string
+	Sign(ctx context.Context) context.Context
+	GetMessageQueues(ctx context.Context, topic string) ([]*v2.MessageQueue, error)
+	GracefulStop() error
+}
+
+type NewClientFunc func(*Config, ...ClientOption) (Client, error)
+
+var _ = Client(&defaultClient{})
+
+type defaultClient struct {
+	config        *Config
+	opts          clientOptions
+	conn          ClientConn
+	msc           v2.MessagingServiceClient
+	router        sync.Map
+	clientID      string
+	clientManager ClientManager
+	done          chan struct{}
+}
+
+func NewClient(config *Config, opts ...ClientOption) (Client, error) {
+	cli := &defaultClient{
+		config:   config,
+		opts:     defaultNSOptions,
+		clientID: fmt.Sprintf("%s@%d@%s", innerMD.Rocketmq, rand.Int(), shortuuid.New()),
+	}
+	for _, opt := range opts {
+		opt.apply(&cli.opts)
+	}
+	conn, err := cli.opts.clientConnFunc(config.Endpoint, cli.opts.connOptions...)
+	if err != nil {
+		return nil, err
+	}
+
+	cli.conn = conn
+	cli.msc = v2.NewMessagingServiceClient(conn.Conn())
+	cli.done = make(chan struct{})
+	cli.startUp()
+	return cli, nil
+}
+
+func (cli *defaultClient) GetClientID() string {
+	return cli.clientID
+}
+
+func (cli *defaultClient) GetMessageQueues(ctx context.Context, topic string) ([]*v2.MessageQueue, error) {
+	item, ok := cli.router.Load(topic)
+	if ok {
+		if ret, ok := item.([]*v2.MessageQueue); ok {
+			return ret, nil
+		}
+	}
+	route, err := cli.queryRoute(ctx, topic)
+	if err != nil {
+		return nil, err
+	}
+	cli.router.Store(topic, route)
+	return route, nil
+}
+
+func (cli *defaultClient) queryRoute(ctx context.Context, topic string) ([]*v2.MessageQueue, error) {
+	ctx = cli.Sign(ctx)
+	response, err := cli.msc.QueryRoute(ctx, cli.getQueryRouteRequest(topic))
+	if err != nil {
+		return nil, err
+	}
+	if response.GetStatus().GetCode() != v2.Code_OK {
+		return nil, fmt.Errorf("QueryRoute err = %s", response.String())
+	}
+
+	if len(response.GetMessageQueues()) == 0 {
+		return nil, errors.New("rocketmq: no available brokers")
+	}
+	return response.GetMessageQueues(), nil
+}
+
+func (cli *defaultClient) getQueryRouteRequest(topic string) *v2.QueryRouteRequest {
+	return &v2.QueryRouteRequest{
+		Topic: &v2.Resource{
+			ResourceNamespace: cli.config.NameSpace,
+			Name:              topic,
+		},
+		Endpoints: cli.parseTarget(cli.conn.Conn().Target()),
+	}
+}
+
+func (cli *defaultClient) parseTarget(target string) *v2.Endpoints {
+	ret := &v2.Endpoints{
+		Scheme: v2.AddressScheme_DOMAIN_NAME,
+		Addresses: []*v2.Address{
+			{
+				Host: "",
+				Port: 80,
+			},
+		},
+	}
+	var (
+		path string
+	)
+	u, err := url.Parse(target)
+	if err != nil {
+		path = target
+		ret.Scheme = v2.AddressScheme_IPv4
+	} else {
+		path = u.Path
+	}
+	paths := strings.Split(path, ":")
+	if len(paths) > 0 {
+		if port, err := strconv.ParseInt(paths[1], 10, 32); err != nil {
+			ret.Addresses[0].Port = int32(port)
+		}
+	}
+	ret.Addresses[0].Host = paths[0]
+	return ret
+}
+
+func (cli *defaultClient) startUp() {
+	log.Printf("Begin to start the rocketmq client, clientId=%s", cli.clientID)
+	cli.clientManager = defaultClientManagerRegistry.RegisterClient(cli)
+	f := func() {
+		cli.router.Range(func(k, v interface{}) bool {
+			ctx, _ := context.WithTimeout(context.TODO(), cli.opts.timeout)
+			item, _ := cli.queryRoute(ctx, k.(string))
+			if !reflect.DeepEqual(item, v) {
+				// cli.router.Store(k, item)
+				// b, ok := cli.brokers.Load(k)
+				// if ok {
+				// 	bo, ok := b.(RpcClient)
+				// 	if ok {
+				// 		bo.SetMessageQueue(item)
+				// 	}
+				// }
+			}
+			return true
+		})
+	}
+	ticker.Tick(f, cli.opts.tickerDuration, cli.done)
+}
+
+func (cli *defaultClient) GracefulStop() error {
+	close(cli.done)
+	cli.done <- struct{}{}
+	// cli.brokers.Range(func(k, v interface{}) bool {
+	// 	broker, ok := v.(RpcClient)
+	// 	if ok {
+	// 		_ = broker.GracefulStop()
+	// 	}
+	// 	return true
+	// })
+	return cli.conn.Close()
+}
+
+func (cli *defaultClient) Sign(ctx context.Context) context.Context {
+	now := time.Now().Format("20060102T150405Z")
+	return metadata.AppendToOutgoingContext(ctx,
+		innerMD.LanguageKey,
+		innerMD.LanguageValue,
+		innerMD.ProtocolKey,
+		innerMD.ProtocolValue,
+		innerMD.RequestID,
+		uuid.New().String(),
+		innerMD.VersionKey,
+		innerMD.VersionValue,
+		// innerMD.NameSpace,
+		// cli.config.NameSpace,
+		innerMD.ClintID,
+		cli.clientID,
+		innerMD.DateTime,
+		now,
+		innerMD.Authorization,
+		fmt.Sprintf("%s %s=%s/%s/%s, %s=%s, %s=%s",
+			innerMD.EncryptHeader,
+			innerMD.Credential,
+			cli.config.Credentials.AccessKey,
+			cli.config.Region,
+			innerMD.Rocketmq,
+			innerMD.SignedHeaders,
+			innerMD.DateTime,
+			innerMD.Signature,
+			func() string {
+				h := hmac.New(sha1.New, []byte(cli.config.Credentials.AccessSecret))
+				h.Write([]byte(now))
+				return hex.EncodeToString(h.Sum(nil))
+			}(),
+		),
+	)
+}
diff --git a/golang/client_manager.go b/golang/client_manager.go
new file mode 100644
index 0000000..b35686a
--- /dev/null
+++ b/golang/client_manager.go
@@ -0,0 +1,224 @@
+package golang
+
+import (
+	// "context"
+	"context"
+	"fmt"
+	"log"
+	"sync"
+	"time"
+
+	"github.com/apache/rocketmq-clients/golang/pkg/ticker"
+	v2 "github.com/apache/rocketmq-clients/golang/protocol/v2"
+	"github.com/valyala/fastrand"
+	// "time"
+	// v2 "github.com/apache/rocketmq-clients/golang/protocol/v2"
+)
+
+type ClientManager interface {
+	RegisterClient(client Client)
+	UnRegisterClient(client Client)
+	// HeartBeat(ctx context.Context, endpoints *v2.Endpoints, req *v2.HeartbeatRequest, duration time.Duration) error
+	SendMessage(ctx context.Context, endpoints *v2.Endpoints, request *v2.SendMessageRequest, duration time.Duration) (*v2.SendMessageResponse, error)
+	// QueryAssignment(ctx context.Context, endpoints *v2.Endpoints, topic string, duration time.Duration) ([]*v2.Assignment, error)
+	// ReceiveMessage(ctx context.Context, endpoints *v2.Endpoints, partition *v2.MessageQueue, topic string, duration time.Duration) (v2.MessagingService_ReceiveMessageClient, error)
+	// AckMessage(ctx context.Context, endpoints *v2.Endpoints, msg *MessageExt, duration time.Duration) error
+}
+
+type clientManagerOptions struct {
+	RPC_CLIENT_MAX_IDLE_DURATION time.Duration
+
+	RPC_CLIENT_IDLE_CHECK_INITIAL_DELAY time.Duration
+	RPC_CLIENT_IDLE_CHECK_PERIOD        time.Duration
+
+	HEART_BEAT_INITIAL_DELAY time.Duration
+	HEART_BEAT_PERIOD        time.Duration
+
+	LOG_STATS_INITIAL_DELAY time.Duration
+	LOG_STATS_PERIOD        time.Duration
+
+	ANNOUNCE_SETTINGS_DELAY  time.Duration
+	ANNOUNCE_SETTINGS_PERIOD time.Duration
+}
+
+var defaultClientManagerOptions = clientManagerOptions{
+	RPC_CLIENT_MAX_IDLE_DURATION: time.Minute * 30,
+
+	RPC_CLIENT_IDLE_CHECK_INITIAL_DELAY: time.Second * 5,
+	RPC_CLIENT_IDLE_CHECK_PERIOD:        time.Minute * 1,
+
+	HEART_BEAT_INITIAL_DELAY: time.Second * 1,
+	HEART_BEAT_PERIOD:        time.Second * 10,
+
+	LOG_STATS_INITIAL_DELAY: time.Second * 60,
+	LOG_STATS_PERIOD:        time.Second * 60,
+
+	ANNOUNCE_SETTINGS_DELAY:  time.Second * 1,
+	ANNOUNCE_SETTINGS_PERIOD: time.Second * 15,
+}
+
+type clientManagerImpl struct {
+	rpcClientTable     map[string]RpcClient
+	rpcClientTableLock sync.RWMutex
+	clientTable        sync.Map
+	done               chan struct{}
+	opts               clientManagerOptions
+}
+
+type ClientManagerRegistry interface {
+	RegisterClient(client Client) ClientManager
+	UnRegisterClient(client Client) bool
+}
+
+type clientManagerRegistry struct {
+	clientIds              map[string]bool
+	clientIdsLock          sync.Mutex
+	singletonClientManager ClientManager
+}
+
+var defaultClientManagerRegistry = &clientManagerRegistry{
+	clientIds: make(map[string]bool),
+}
+
+var _ = ClientManagerRegistry(&clientManagerRegistry{})
+
+func (cmr *clientManagerRegistry) RegisterClient(client Client) ClientManager {
+	cmr.clientIdsLock.Lock()
+	defer cmr.clientIdsLock.Unlock()
+
+	if cmr.singletonClientManager == nil {
+		cmr.singletonClientManager = NewClientManagerImpl()
+		cmr.singletonClientManager.(*clientManagerImpl).startUp()
+	}
+	cmr.clientIds[client.GetClientID()] = true
+	cmr.singletonClientManager.RegisterClient(client)
+	return cmr.singletonClientManager
+}
+
+func (cmr *clientManagerRegistry) UnRegisterClient(client Client) bool {
+	var tmpClientManager ClientManager
+
+	cmr.clientIdsLock.Lock()
+	{
+		delete(cmr.clientIds, client.GetClientID())
+		cmr.singletonClientManager.UnRegisterClient(client)
+		if len(cmr.clientIds) == 0 {
+			tmpClientManager = cmr.singletonClientManager
+			cmr.singletonClientManager = nil
+		}
+	}
+	cmr.clientIdsLock.Unlock()
+	if tmpClientManager != nil {
+		tmpClientManager.(*clientManagerImpl).shutdown()
+	}
+	return tmpClientManager != nil
+}
+
+var _ = ClientManager(&clientManagerImpl{})
+
+func NewClientManagerImpl() *clientManagerImpl {
+	return &clientManagerImpl{
+		rpcClientTable: make(map[string]RpcClient),
+		done:           make(chan struct{}),
+		opts:           defaultClientManagerOptions,
+	}
+}
+func (cm *clientManagerImpl) RegisterClient(client Client) {
+	cm.clientTable.Store(client.GetClientID(), client)
+}
+
+func (cm *clientManagerImpl) UnRegisterClient(client Client) {
+	cm.clientTable.Delete(client.GetClientID())
+}
+
+func (cm *clientManagerImpl) startUp() {
+	log.Println("Begin to start the client manager")
+
+	f := func() {
+		time.Sleep(cm.opts.RPC_CLIENT_IDLE_CHECK_INITIAL_DELAY)
+		cm.clearIdleRpcClients()
+	}
+	ticker.Tick(f, (cm.opts.RPC_CLIENT_IDLE_CHECK_PERIOD), cm.done)
+
+	f1 := func() {
+		time.Sleep(cm.opts.HEART_BEAT_INITIAL_DELAY)
+		cm.doHeartbeat()
+	}
+	ticker.Tick(f1, (cm.opts.HEART_BEAT_PERIOD), cm.done)
+
+	f2 := func() {
+		time.Sleep(cm.opts.LOG_STATS_INITIAL_DELAY)
+		cm.doStats()
+	}
+	ticker.Tick(f2, (cm.opts.LOG_STATS_PERIOD), cm.done)
+
+	f3 := func() {
+		time.Sleep(cm.opts.ANNOUNCE_SETTINGS_DELAY)
+		cm.syncSettings()
+	}
+	ticker.Tick(f3, (cm.opts.ANNOUNCE_SETTINGS_PERIOD), cm.done)
+
+	log.Println("The client manager starts successfully")
+}
+func (cm *clientManagerImpl) clearIdleRpcClients() {
+	log.Println("clearIdleRpcClients")
+}
+func (cm *clientManagerImpl) doHeartbeat() {
+	log.Println("doHeartbeat")
+}
+func (cm *clientManagerImpl) doStats() {
+	log.Println("doStats")
+}
+func (cm *clientManagerImpl) syncSettings() {
+	log.Println("syncSettings")
+}
+func (cm *clientManagerImpl) shutdown() {
+	log.Println("Begin to shutdown the client manager")
+	close(cm.done)
+	cm.done <- struct{}{}
+	cm.cleanRpcClient()
+	log.Println("Shutdown the client manager successfully")
+}
+func (cm *clientManagerImpl) cleanRpcClient() {
+	log.Println("cleanRpcClient")
+}
+func (cm *clientManagerImpl) getRpcClient(endpoints *v2.Endpoints) (RpcClient, error) {
+	addresses := endpoints.GetAddresses()
+	idx := fastrand.Uint32n(uint32(len(addresses)))
+	selectAddress := addresses[idx]
+	target := fmt.Sprintf("%s:%d", selectAddress.Host, selectAddress.Port)
+
+	cm.rpcClientTableLock.RLock()
+	item, ok := cm.rpcClientTable[target]
+	cm.rpcClientTableLock.RUnlock()
+	if ok {
+		if ret, ok := item.(*rpcClient); ok {
+			return ret, nil
+		}
+	}
+
+	cm.rpcClientTableLock.Lock()
+	defer cm.rpcClientTableLock.Unlock()
+
+	// double check
+	item, ok = cm.rpcClientTable[target]
+	if ok {
+		if ret, ok := item.(*rpcClient); ok {
+			return ret, nil
+		}
+	}
+	rpcClient, err := NewRpcClient(target)
+	if err != nil {
+		return nil, err
+	}
+	cm.rpcClientTable[target] = rpcClient
+	return rpcClient, nil
+}
+
+func (cm *clientManagerImpl) SendMessage(ctx context.Context, endpoints *v2.Endpoints, request *v2.SendMessageRequest, duration time.Duration) (*v2.SendMessageResponse, error) {
+	rpcClient, err := cm.getRpcClient(endpoints)
+	if err != nil {
+		return nil, err
+	}
+	return rpcClient.SendMessage(ctx, request, duration)
+}
diff --git a/golang/ns_options.go b/golang/client_options.go
similarity index 51%
rename from golang/ns_options.go
rename to golang/client_options.go
index 072079f..4b528b6 100644
--- a/golang/ns_options.go
+++ b/golang/client_options.go
@@ -21,38 +21,36 @@ import (
 	"time"
 )
 
-type nameServerOptions struct {
-	tickerDuration time.Duration
-	timeout        time.Duration
-	clientConnFunc ClientConnFunc
-	connOptions    []ConnOption
-	brokerFunc     BrokerFunc
-	brokerOptions  []BrokerOption
+type clientOptions struct {
+	tickerDuration   time.Duration
+	timeout          time.Duration
+	clientConnFunc   ClientConnFunc
+	connOptions      []ConnOption
+	rpcClientOptions []RpcClientOption
 }
 
-var defaultNSOptions = nameServerOptions{
+var defaultNSOptions = clientOptions{
 	tickerDuration: time.Second * 30,
 	timeout:        time.Millisecond * 3000,
 	clientConnFunc: NewClientConn,
-	brokerFunc:     NewBroker,
 }
 
-// A NameServerOption sets options such as timeout, etc.
-type NameServerOption interface {
-	apply(*nameServerOptions)
+// A ClientOption sets options such as timeout, etc.
+type ClientOption interface {
+	apply(*clientOptions)
 }
 
 // funcNSOption wraps a function that modifies options into an implementation of
-// the NameServerOption interface.
+// the ClientOption interface.
 type funcNSOption struct {
-	f func(options *nameServerOptions)
+	f func(options *clientOptions)
 }
 
-func (fpo *funcNSOption) apply(po *nameServerOptions) {
+func (fpo *funcNSOption) apply(po *clientOptions) {
 	fpo.f(po)
 }
 
-func newFuncNSOption(f func(options *nameServerOptions)) *funcNSOption {
+func newFuncNSOption(f func(options *clientOptions)) *funcNSOption {
 	return &funcNSOption{
 		f: f,
 	}
@@ -60,46 +58,38 @@ func newFuncNSOption(f func(options *nameServerOptions)) *funcNSOption {
 
 // WithQueryRouteTimeout returns a Option that sets timeout duration for nameserver.
 // Default is 3s.
-func WithQueryRouteTimeout(d time.Duration) NameServerOption {
-	return newFuncNSOption(func(o *nameServerOptions) {
+func WithQueryRouteTimeout(d time.Duration) ClientOption {
+	return newFuncNSOption(func(o *clientOptions) {
 		o.timeout = d
 	})
 }
 
 // WithTickerDuration returns a Option that sets ticker duration for nameserver.
 // Default is 30s.
-func WithTickerDuration(d time.Duration) NameServerOption {
-	return newFuncNSOption(func(o *nameServerOptions) {
+func WithTickerDuration(d time.Duration) ClientOption {
+	return newFuncNSOption(func(o *clientOptions) {
 		o.tickerDuration = d
 	})
 }
 
 // WithClientConnFunc returns a Option that sets ClientConnFunc for nameserver.
 // Default is NewClientConn.
-func WithClientConnFunc(f ClientConnFunc) NameServerOption {
-	return newFuncNSOption(func(o *nameServerOptions) {
+func WithClientConnFunc(f ClientConnFunc) ClientOption {
+	return newFuncNSOption(func(o *clientOptions) {
 		o.clientConnFunc = f
 	})
 }
 
-// WithBrokerFunc returns a Option that sets BrokerFunc for nameserver.
-// Default is NewClient.
-func WithBrokerFunc(f BrokerFunc) NameServerOption {
-	return newFuncNSOption(func(o *nameServerOptions) {
-		o.brokerFunc = f
-	})
-}
-
 // WithConnOptions returns a Option that sets ConnOption for grpc ClientConn.
-func WithConnOptions(opts ...ConnOption) NameServerOption {
-	return newFuncNSOption(func(o *nameServerOptions) {
+func WithConnOptions(opts ...ConnOption) ClientOption {
+	return newFuncNSOption(func(o *clientOptions) {
 		o.connOptions = append(o.connOptions, opts...)
 	})
 }
 
-// WithBrokerOptions returns a Option that sets BrokerOption for grpc ClientConn.
-func WithBrokerOptions(opts ...BrokerOption) NameServerOption {
-	return newFuncNSOption(func(o *nameServerOptions) {
-		o.brokerOptions = append(o.brokerOptions, opts...)
+// WithRpcClientOptions returns a Option that sets RpcClientOption for grpc ClientConn.
+func WithRpcClientOptions(opts ...RpcClientOption) ClientOption {
+	return newFuncNSOption(func(o *clientOptions) {
+		o.rpcClientOptions = append(o.rpcClientOptions, opts...)
 	})
 }
diff --git a/golang/config.go b/golang/config.go
index 04cbf5c..4089e31 100644
--- a/golang/config.go
+++ b/golang/config.go
@@ -20,9 +20,9 @@ package golang
 import "github.com/apache/rocketmq-clients/golang/credentials"
 
 type Config struct {
-	Endpoint    string                          `validate:"required"`
-	Region      string                          `validate:"required"`
-	NameSpace   string                          `validate:"required"`
+	Endpoint    string `validate:"required"`
+	Region      string `validate:"required"`
+	NameSpace   string
 	Group       string                          `validate:"required"`
 	Credentials *credentials.SessionCredentials `validate:"required"`
 }
diff --git a/golang/conn.go b/golang/conn.go
index 14ddaae..15710d5 100644
--- a/golang/conn.go
+++ b/golang/conn.go
@@ -19,33 +19,25 @@ package golang
 
 import (
 	"context"
-	"crypto/hmac"
-	"crypto/sha1"
-	"encoding/hex"
 	"errors"
 	"fmt"
-	"time"
 
-	innerMD "github.com/apache/rocketmq-clients/golang/metadata"
 	"github.com/apache/rocketmq-clients/golang/pkg/grpc/middleware/zaplog"
 	validator "github.com/go-playground/validator/v10"
-	"github.com/google/uuid"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/credentials"
 	"google.golang.org/grpc/keepalive"
-	"google.golang.org/grpc/metadata"
 )
 
 var (
 	ErrNoAvailableEndpoints = errors.New("rocketmq: no available endpoints")
 )
 
-type ClientConnFunc func(*Config, ...ConnOption) (ClientConn, error)
+type ClientConnFunc func(string, ...ConnOption) (ClientConn, error)
 
 type ClientConn interface {
 	Conn() *grpc.ClientConn
 	Close() error
-	Config() *Config
 }
 
 var _ = ClientConn(&clientConn{})
@@ -57,22 +49,17 @@ type clientConn struct {
 	cancel   context.CancelFunc
 	callOpts []grpc.CallOption
 	conn     *grpc.ClientConn
-	config   *Config
 	validate *validator.Validate
 }
 
-func NewClientConn(config *Config, opts ...ConnOption) (ClientConn, error) {
+func NewClientConn(endpoint string, opts ...ConnOption) (ClientConn, error) {
 	client := &clientConn{
 		opts:     defaultConnOptions,
-		config:   config,
 		validate: validator.New(),
 	}
-	if client.config == nil {
+	if len(endpoint) == 0 {
 		return nil, ErrNoAvailableEndpoints
 	}
-	if err := client.validate.Struct(client.config); err != nil {
-		return nil, err
-	}
 	for _, opt := range opts {
 		opt.apply(&client.opts)
 	}
@@ -100,7 +87,7 @@ func NewClientConn(config *Config, opts ...ConnOption) (ClientConn, error) {
 		}
 	}
 
-	conn, err := client.dial(config.Endpoint)
+	conn, err := client.dial(endpoint)
 	if err != nil {
 		client.cancel()
 		return nil, err
@@ -119,10 +106,6 @@ func (c *clientConn) Close() error {
 	return c.conn.Close()
 }
 
-func (c *clientConn) Config() *Config {
-	return c.config
-}
-
 func (c *clientConn) dialSetupOpts(dopts ...grpc.DialOption) (opts []grpc.DialOption, err error) {
 	if c.opts.DialKeepAliveTime > 0 {
 		opts = append(opts, grpc.WithKeepaliveParams(keepalive.ClientParameters{
@@ -137,7 +120,6 @@ func (c *clientConn) dialSetupOpts(dopts ...grpc.DialOption) (opts []grpc.DialOp
 	}
 	opts = append(opts, grpc.WithBlock(), grpc.WithChainUnaryInterceptor(
 		zaplog.UnaryClientInterceptor(c.opts.Logger),
-		c.UnaryClientInterceptor(),
 	))
 	return
 }
@@ -158,44 +140,3 @@ func (c *clientConn) dial(target string, dopts ...grpc.DialOption) (*grpc.Client
 	}
 	return conn, nil
 }
-
-func (c *clientConn) UnaryClientInterceptor() grpc.UnaryClientInterceptor {
-	return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
-		newCtx := c.sign(ctx)
-		return invoker(newCtx, method, req, reply, cc, opts...)
-	}
-}
-
-func (c *clientConn) sign(ctx context.Context) context.Context {
-	now := time.Now().Format("20060102T150405Z")
-	return metadata.AppendToOutgoingContext(ctx,
-		innerMD.LanguageKey,
-		innerMD.LanguageValue,
-		innerMD.ProtocolKey,
-		innerMD.ProtocolValue,
-		innerMD.RequestID,
-		uuid.New().String(),
-		innerMD.VersionKey,
-		innerMD.VersionValue,
-		innerMD.NameSpace,
-		c.config.NameSpace,
-		innerMD.DateTime,
-		now,
-		innerMD.Authorization,
-		fmt.Sprintf("%s %s=%s/%s/%s, %s=%s, %s=%s",
-			innerMD.EncryptHeader,
-			innerMD.Credential,
-			c.config.Credentials.AccessKey,
-			c.config.Region,
-			innerMD.Rocketmq,
-			innerMD.SignedHeaders,
-			innerMD.DateTime,
-			innerMD.Signature,
-			func() string {
-				h := hmac.New(sha1.New, []byte(c.config.Credentials.AccessSecret))
-				h.Write([]byte(now))
-				return hex.EncodeToString(h.Sum(nil))
-			}(),
-		),
-	)
-}
diff --git a/golang/consumer.go b/golang/consumer.go
index 8c5f02e..3f1d91e 100644
--- a/golang/consumer.go
+++ b/golang/consumer.go
@@ -17,137 +17,161 @@
 
 package golang
 
-import (
-	"context"
-	"reflect"
-	"sync"
-	"time"
+// import (
+// 	"context"
+// 	"io"
+// 	"reflect"
+// 	"sync"
+// 	"time"
 
-	"github.com/apache/rocketmq-clients/golang/pkg/ticker"
-	v1 "github.com/apache/rocketmq-clients/golang/protocol/v1"
-)
+// 	"github.com/apache/rocketmq-clients/golang/pkg/ticker"
+// 	v2 "github.com/apache/rocketmq-clients/golang/protocol/v2"
+// )
 
-type Consumer interface {
-	Start() error
-	Consume(ctx context.Context, topic string, handler Handler) error
-	GracefulStop() error
-}
+// type Consumer interface {
+// 	Start() error
+// 	Consume(ctx context.Context, topic string, handler Handler) error
+// 	GracefulStop() error
+// }
 
-var _ = Consumer(&consumer{})
+// var _ = Consumer(&consumer{})
 
-type consumer struct {
-	opts       consumerOptions
-	ns         NameServer
-	assignment sync.Map
-	handlers   map[string]Handler
-	done       chan struct{}
-}
+// type consumer struct {
+// 	opts       consumerOptions
+// 	ns         NameServer
+// 	assignment sync.Map
+// 	handlers   map[string]Handler
+// 	done       chan struct{}
+// }
 
-type Handler func(ctx context.Context, entry *MessageExt) error
+// type Handler func(ctx context.Context, entry *MessageExt) error
 
-func NewConsumer(config *Config, opts ...ConsumerOption) (Consumer, error) {
-	ns, err := NewNameServer(config)
-	if err != nil {
-		return nil, err
-	}
-	c := &consumer{
-		opts:     defaultConsumerOptions,
-		handlers: make(map[string]Handler),
-		done:     make(chan struct{}),
-	}
-	for _, opt := range opts {
-		opt.apply(&c.opts)
-	}
+// func NewConsumer(config *Config, opts ...ConsumerOption) (Consumer, error) {
+// 	ns, err := NewNameServer(config)
+// 	if err != nil {
+// 		return nil, err
+// 	}
+// 	c := &consumer{
+// 		opts:     defaultConsumerOptions,
+// 		handlers: make(map[string]Handler),
+// 		done:     make(chan struct{}),
+// 	}
+// 	for _, opt := range opts {
+// 		opt.apply(&c.opts)
+// 	}
 
-	c.ns = ns
-	return c, nil
-}
+// 	c.ns = ns
+// 	return c, nil
+// }
 
-func (c *consumer) Start() error {
-	c.assignment.Range(func(k, v interface{}) bool {
-		topic, ok := k.(string)
-		if ok {
-			handler := c.handlers[topic]
-			f := func() {
-				ctx, _ := context.WithTimeout(context.TODO(), time.Second*20)
-				b, err := c.ns.GetBroker(ctx, topic)
-				if err != nil {
-					return
-				}
-				assign, ok := v.([]*v1.Assignment)
-				if !ok || len(assign) == 0 {
-					return
-				}
-				var wg sync.WaitGroup
-				for i := 0; i < len(assign); i++ {
-					wg.Add(1)
-					i := i
-					go func() {
-						defer wg.Done()
-						messages, err := b.ReceiveMessage(ctx, assign[i].Partition, topic)
-						if err != nil {
-							return
-						}
-						for _, msg := range messages {
-							if err = handler(ctx, msg); err == nil {
-								_ = b.AckMessage(ctx, msg)
-							}
-						}
-					}()
-				}
-				wg.Wait()
-			}
-			ticker.Tick(f, time.Second*20, c.done)
-		}
-		return true
-	})
-	<-c.done
-	return nil
-}
+// func (c *consumer) Start() error {
+// 	c.assignment.Range(func(k, v interface{}) bool {
+// 		topic, ok := k.(string)
+// 		if ok {
+// 			handler := c.handlers[topic]
+// 			f := func() {
+// 				ctx, _ := context.WithTimeout(context.TODO(), time.Second*20)
+// 				b, err := c.ns.GetBroker(ctx, topic)
+// 				if err != nil {
+// 					return
+// 				}
+// 				assign, ok := v.([]*v2.Assignment)
+// 				if !ok || len(assign) == 0 {
+// 					return
+// 				}
+// 				var wg sync.WaitGroup
+// 				for i := 0; i < len(assign); i++ {
+// 					wg.Add(1)
+// 					i := i
+// 					go func() {
+// 						defer wg.Done()
+// 						stream, err := b.ReceiveMessage(ctx, assign[i].MessageQueue, topic)
+// 						if err != nil {
+// 							return
+// 						}
+// 						for {
+// 							resp, err := stream.Recv()
+// 							if err == io.EOF {
+// 								break
+// 							}
+// 							if err != nil {
+// 								time.Sleep(time.Second * 5)
+// 								continue
+// 							}
+// 							msg, ok := resp.GetContent().(*v2.ReceiveMessageResponse_Message)
+// 							if !ok {
+// 								continue
+// 							}
+// 							messageExt := &MessageExt{
+// 								MessageID:     msg.Message.GetSystemProperties().GetMessageId(),
+// 								ReceiptHandle: msg.Message.GetSystemProperties().GetReceiptHandle(),
+// 								Message: Message{
+// 									Topic:      msg.Message.GetTopic().GetName(),
+// 									Body:       msg.Message.GetBody(),
+// 									Keys:       msg.Message.GetSystemProperties().GetKeys(),
+// 									Tag:        msg.Message.GetSystemProperties().GetTag(),
+// 									Properties: msg.Message.GetUserProperties(),
+// 								},
+// 							}
+// 							if err = handler(ctx, messageExt); err == nil {
+// 								_ = b.AckMessage(ctx, messageExt)
+// 							}
+// 						}
+// 					}()
+// 				}
+// 				wg.Wait()
+// 			}
+// 			ticker.Tick(f, time.Second*20, c.done)
+// 		}
+// 		return true
+// 	})
+// 	<-c.done
+// 	return nil
+// }
 
-func (c *consumer) Consume(ctx context.Context, topic string, handler Handler) error {
-	if err := c.queryAssignment(ctx, topic); err != nil {
-		return err
-	}
+// func (c *consumer) Consume(ctx context.Context, topic string, handler Handler) error {
+// 	if err := c.queryAssignment(ctx, topic); err != nil {
+// 		return err
+// 	}
 
-	c.handlers[topic] = handler
-	return nil
-}
+// 	c.handlers[topic] = handler
+// 	return nil
+// }
 
-func (c *consumer) queryAssignment(ctx context.Context, topic string) error {
-	b, err := c.ns.GetBroker(ctx, topic)
-	if err != nil {
-		return err
-	}
-	assignment, err := b.QueryAssignment(ctx, topic)
-	if err != nil {
-		return err
-	}
-	c.assignment.Store(topic, assignment)
-	f := func() {
-		assign, err := b.QueryAssignment(ctx, topic)
-		if err != nil {
-			return
-		}
-		cache, ok := c.assignment.Load(topic)
-		if !ok {
-			return
-		}
-		oldAssign, ok := cache.([]*v1.Assignment)
-		if !ok {
-			return
-		}
-		if reflect.DeepEqual(assign, oldAssign) {
-			return
-		}
-		c.assignment.Store(topic, assign)
-	}
-	ticker.Tick(f, time.Second*10, c.done)
-	return nil
-}
+// func (c *consumer) queryAssignment(ctx context.Context, topic string) error {
+// 	b, err := c.ns.GetBroker(ctx, topic)
+// 	if err != nil {
+// 		return err
+// 	}
+// 	assignment, err := b.QueryAssignment(ctx, topic)
+// 	if err != nil {
+// 		return err
+// 	}
+// 	c.assignment.Store(topic, assignment)
+// 	f := func() {
+// 		assign, err := b.QueryAssignment(ctx, topic)
+// 		if err != nil {
+// 			return
+// 		}
+// 		cache, ok := c.assignment.Load(topic)
+// 		if !ok {
+// 			return
+// 		}
+// 		oldAssign, ok := cache.([]*v2.Assignment)
+// 		if !ok {
+// 			return
+// 		}
+// 		if reflect.DeepEqual(assign, oldAssign) {
+// 			return
+// 		}
+// 		c.assignment.Store(topic, assign)
+// 	}
+// 	ticker.Tick(f, time.Second*10, c.done)
+// 	return nil
+// }
 
-func (c *consumer) GracefulStop() error {
-	defer close(c.done)
-	c.done <- struct{}{}
-	return c.ns.GracefulStop()
-}
+// func (c *consumer) GracefulStop() error {
+// 	defer close(c.done)
+// 	c.done <- struct{}{}
+// 	return c.ns.GracefulStop()
+// }
diff --git a/golang/example/consumer/main.go b/golang/example/consumer/main.go
index c44fbf4..70e08e6 100644
--- a/golang/example/consumer/main.go
+++ b/golang/example/consumer/main.go
@@ -17,44 +17,44 @@
 
 package main
 
-import (
-	"context"
-	"fmt"
-	"log"
+// import (
+// 	"context"
+// 	"fmt"
+// 	"log"
 
-	"github.com/apache/rocketmq-clients/golang"
-	"github.com/apache/rocketmq-clients/golang/credentials"
-)
+// 	"github.com/apache/rocketmq-clients/golang"
+// 	"github.com/apache/rocketmq-clients/golang/credentials"
+// )
 
-const (
-	Topic         = "LIPPI_DOC_TEST"
-	ConsumerGroup = "GID_LIPPI_DOC_TEST"
-	NameSpace     = "MQ_INST_1487434140287185_BYAzh6Mc"
-	Endpoint      = "116.62.231.199:80"
-	AccessKey     = "xxxx"
-	SecretKey     = "xxxx"
-)
+// const (
+// 	Topic         = "v2_grpc"
+// 	ConsumerGroup = "v2_grpc"
+// 	NameSpace     = ""
+// 	Endpoint      = "121.196.167.124:8081"
+// 	AccessKey     = ""
+// 	SecretKey     = ""
+// )
 
-func main() {
-	consumer, err := golang.NewConsumer(&golang.Config{
-		Endpoint:  Endpoint,
-		NameSpace: NameSpace,
-		Group:     ConsumerGroup,
-		Region:    "cn-zhangjiakou",
-		Credentials: &credentials.SessionCredentials{
-			AccessKey:    AccessKey,
-			AccessSecret: SecretKey,
-		},
-	})
-	if err != nil {
-		log.Fatal(err)
-	}
-	defer consumer.GracefulStop()
-	if err := consumer.Consume(context.TODO(), Topic, func(ctx context.Context, entry *golang.MessageExt) error {
-		fmt.Printf("%#v\n", entry)
-		return nil
-	}); err != nil {
-		log.Fatal(err)
-	}
-	consumer.Start()
-}
+// func main() {
+// 	consumer, err := golang.NewConsumer(&golang.Config{
+// 		Endpoint:  Endpoint,
+// 		NameSpace: NameSpace,
+// 		Group:     ConsumerGroup,
+// 		Region:    "cn-zhangjiakou",
+// 		Credentials: &credentials.SessionCredentials{
+// 			AccessKey:    AccessKey,
+// 			AccessSecret: SecretKey,
+// 		},
+// 	})
+// 	if err != nil {
+// 		log.Fatal(err)
+// 	}
+// 	defer consumer.GracefulStop()
+// 	if err := consumer.Consume(context.TODO(), Topic, func(ctx context.Context, entry *golang.MessageExt) error {
+// 		fmt.Printf("%#v\n", entry)
+// 		return nil
+// 	}); err != nil {
+// 		log.Fatal(err)
+// 	}
+// 	consumer.Start()
+// }
diff --git a/golang/example/producer/main.go b/golang/example/producer/main.go
index 6722515..08392ee 100644
--- a/golang/example/producer/main.go
+++ b/golang/example/producer/main.go
@@ -29,20 +29,19 @@ import (
 )
 
 const (
-	Topic         = "LIPPI_DOC_TEST"
-	ConsumerGroup = "GID_LIPPI_DOC_TEST"
-	NameSpace     = "MQ_INST_1487434140287185_BYAzh6Mc"
-	Endpoint      = "116.62.231.199:80"
-	AccessKey     = "xxx"
-	SecretKey     = "xxxx"
+	Topic         = "v2_grpc"
+	ConsumerGroup = "v2_grpc"
+	NameSpace     = ""
+	Endpoint      = "121.196.167.124:8081"
+	AccessKey     = ""
+	SecretKey     = ""
 )
 
 func main() {
 	producer, err := golang.NewProducer(&golang.Config{
-		Endpoint:  Endpoint,
-		NameSpace: NameSpace,
-		Group:     ConsumerGroup,
-		Region:    "cn-zhangjiakou",
+		Endpoint: Endpoint,
+		Group:    ConsumerGroup,
+		Region:   "cn-zhangjiakou",
 		Credentials: &credentials.SessionCredentials{
 			AccessKey:    AccessKey,
 			AccessSecret: SecretKey,
@@ -53,16 +52,30 @@ func main() {
 	}
 	defer producer.GracefulStop()
 	for i := 0; i < 10; i++ {
-		resp, err := producer.Send(context.TODO(), &golang.Message{
+		msg := &golang.Message{
 			Topic: Topic,
 			Body:  []byte(strconv.Itoa(i)),
 			Tag:   "*",
-		})
+		}
+		// msg.SetDelayTimeLevel(time.Now().Add(time.Second * 10))
+		resp, err := producer.Send(context.TODO(), msg)
 		if err != nil {
 			log.Println(err)
-			return
+			// return
+		}
+		for i := 0; i < len(resp); i++ {
+			fmt.Printf("%#v\n", resp[i])
 		}
-		fmt.Printf("%#v\n", resp)
+
+		// producer.SendAsync(context.Background(), msg, func(ctx context.Context, resp []*golang.SendReceipt, err error) {
+		// 	if err != nil {
+		// 		log.Println(err)
+		// 		return
+		// 	}
+		// 	for i := 0; i < len(resp); i++ {
+		// 		fmt.Printf("%#v\n", resp[i])
+		// 	}
+		// })
 		time.Sleep(time.Second * 4)
 	}
 	select {}
diff --git a/golang/go.mod b/golang/go.mod
index 668381f..83a8824 100644
--- a/golang/go.mod
+++ b/golang/go.mod
@@ -4,12 +4,10 @@ go 1.17
 
 require (
 	github.com/go-playground/validator/v10 v10.11.0
-	github.com/golang/protobuf v1.5.2
 	github.com/google/uuid v1.3.0
 	github.com/lithammer/shortuuid/v4 v4.0.0
 	github.com/natefinch/lumberjack v2.0.0+incompatible
 	go.uber.org/zap v1.21.0
-	google.golang.org/genproto v0.0.0-20220623142657-077d458a5694
 	google.golang.org/grpc v1.47.0
 	google.golang.org/protobuf v1.28.0
 )
@@ -17,12 +15,15 @@ require (
 require (
 	github.com/go-playground/locales v0.14.0 // indirect
 	github.com/go-playground/universal-translator v0.18.0 // indirect
+	github.com/golang/protobuf v1.5.2 // indirect
 	github.com/leodido/go-urn v1.2.1 // indirect
+	github.com/valyala/fastrand v1.1.0
 	go.uber.org/atomic v1.7.0 // indirect
 	go.uber.org/multierr v1.6.0 // indirect
 	golang.org/x/crypto v0.0.0-20211215153901-e495a2d5b3d3 // indirect
 	golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2 // indirect
 	golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069 // indirect
 	golang.org/x/text v0.3.7 // indirect
+	google.golang.org/genproto v0.0.0-20220623142657-077d458a5694 // indirect
 	gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect
 )
diff --git a/golang/go.sum b/golang/go.sum
index 14c6d90..8ba2530 100644
--- a/golang/go.sum
+++ b/golang/go.sum
@@ -2,6 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT
 cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
 github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
 github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
+github.com/NebulousLabs/fastrand v0.0.0-20181203155948-6fb6489aac4e h1:n+DcnTNkQnHlwpsrHoQtkrJIO7CBx029fw6oR4vIob4=
+github.com/NebulousLabs/fastrand v0.0.0-20181203155948-6fb6489aac4e/go.mod h1:Bdzq+51GR4/0DIhaICZEOm+OHvXGwwB2trKZ8B4Y6eQ=
 github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
 github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8=
 github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
@@ -88,6 +90,8 @@ github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5
 github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
 github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY=
 github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/valyala/fastrand v1.1.0 h1:f+5HkLW4rsgzdNoleUOB69hyT9IlD2ZQh9GyDMfb5G8=
+github.com/valyala/fastrand v1.1.0/go.mod h1:HWqCzkrkg6QXT8V2EXWvXCoow7vLwOFN002oeRzjapQ=
 github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
 go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI=
 go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw=
diff --git a/golang/message.go b/golang/message.go
index 928c212..1ddac00 100644
--- a/golang/message.go
+++ b/golang/message.go
@@ -17,12 +17,17 @@
 
 package golang
 
+import "time"
+
 type Message struct {
 	Topic      string
 	Body       []byte
 	Tag        string
 	Keys       []string
 	Properties map[string]string
+
+	deliveryTimestamp time.Time
+	messageGroup      string
 }
 
 type MessageExt struct {
@@ -34,3 +39,19 @@ type MessageExt struct {
 type SendReceipt struct {
 	MessageID string
 }
+
+func (msg *Message) SetDelayTimeLevel(deliveryTimestamp time.Time) {
+	msg.deliveryTimestamp = deliveryTimestamp
+}
+
+func (msg *Message) GetDeliveryTimestamp() time.Time {
+	return msg.deliveryTimestamp
+}
+
+func (msg *Message) SetMessageGroup(messageGroup string) {
+	msg.messageGroup = messageGroup
+}
+
+func (msg *Message) GetMessageGroup() string {
+	return msg.messageGroup
+}
diff --git a/golang/metadata/metadata.go b/golang/metadata/metadata.go
index b7b3b6c..2a40d46 100644
--- a/golang/metadata/metadata.go
+++ b/golang/metadata/metadata.go
@@ -18,18 +18,19 @@
 package metadata
 
 const (
-	LanguageKey   = "x-mq-language"
-	ProtocolKey   = "x-mq-protocol"
-	RequestID     = "x-mq-request-id"
-	VersionKey    = "x-mq-client-version"
-	NameSpace     = "x-mq-namespace"
+	LanguageKey = "x-mq-language"
+	ProtocolKey = "x-mq-protocol"
+	RequestID   = "x-mq-request-id"
+	VersionKey  = "x-mq-client-version"
+	// NameSpace     = "x-mq-namespace"
 	DateTime      = "x-mq-date-time"
+	ClintID       = "x-mq-client-id"
 	Authorization = "authorization"
 )
 
 const (
 	LanguageValue = "GO"
-	ProtocolValue = "v1"
+	ProtocolValue = "v2"
 	VersionValue  = "5.0.0"
 )
 
diff --git a/golang/ns.go b/golang/ns.go
deleted file mode 100644
index b2672fc..0000000
--- a/golang/ns.go
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package golang
-
-import (
-	"context"
-	"errors"
-	"fmt"
-	"net/url"
-	"reflect"
-	"strconv"
-	"strings"
-	"sync"
-
-	"github.com/apache/rocketmq-clients/golang/pkg/ticker"
-	v1 "github.com/apache/rocketmq-clients/golang/protocol/v1"
-	"google.golang.org/grpc/codes"
-)
-
-type NameServer interface {
-	GetBroker(ctx context.Context, topic string) (Broker, error)
-	GracefulStop() error
-}
-
-type NewNameServerFunc func(*Config, ...NameServerOption) (NameServer, error)
-
-var _ = NameServer(&defaultNameServer{})
-
-type defaultNameServer struct {
-	opts    nameServerOptions
-	conn    ClientConn
-	msc     v1.MessagingServiceClient
-	router  sync.Map
-	brokers sync.Map
-	done    chan struct{}
-}
-
-func NewNameServer(config *Config, opts ...NameServerOption) (NameServer, error) {
-	ns := &defaultNameServer{
-		opts: defaultNSOptions,
-	}
-	for _, opt := range opts {
-		opt.apply(&ns.opts)
-	}
-	conn, err := ns.opts.clientConnFunc(config, ns.opts.connOptions...)
-	if err != nil {
-		return nil, err
-	}
-
-	ns.conn = conn
-	ns.msc = v1.NewMessagingServiceClient(conn.Conn())
-	ns.done = make(chan struct{})
-	ns.sync()
-	return ns, nil
-}
-
-func (ns *defaultNameServer) GetBroker(ctx context.Context, topic string) (Broker, error) {
-	item, ok := ns.brokers.Load(topic)
-	if ok {
-		if ret, ok := item.(Broker); ok {
-			return ret, nil
-		}
-	}
-	route, err := ns.queryRoute(ctx, topic)
-	if err != nil {
-		return nil, err
-	}
-	ns.router.Store(topic, route)
-	b, err := ns.opts.brokerFunc(ns.conn.Config(), route, ns.opts.brokerOptions...)
-	if err != nil {
-		return nil, err
-	}
-	ns.brokers.Store(topic, b)
-	return b, nil
-}
-
-func (ns *defaultNameServer) queryRoute(ctx context.Context, topic string) ([]*v1.Partition, error) {
-	response, err := ns.msc.QueryRoute(ctx, ns.getQueryRouteRequest(topic))
-	if err != nil {
-		return nil, err
-	}
-	if response.GetCommon().GetStatus().GetCode() != int32(codes.OK) {
-		return nil, fmt.Errorf("QueryRoute err = %s", response.String())
-	}
-
-	if len(response.Partitions) == 0 {
-		return nil, errors.New("rocketmq: no available brokers")
-	}
-	return response.Partitions, nil
-}
-
-func (ns *defaultNameServer) getQueryRouteRequest(topic string) *v1.QueryRouteRequest {
-	return &v1.QueryRouteRequest{
-		Topic: &v1.Resource{
-			ResourceNamespace: ns.conn.Config().NameSpace,
-			Name:              topic,
-		},
-		Endpoints: ns.parseTarget(ns.conn.Conn().Target()),
-	}
-}
-
-func (ns *defaultNameServer) parseTarget(target string) *v1.Endpoints {
-	ret := &v1.Endpoints{
-		Scheme: v1.AddressScheme_DOMAIN_NAME,
-		Addresses: []*v1.Address{
-			{
-				Host: "",
-				Port: 80,
-			},
-		},
-	}
-	var (
-		path string
-	)
-	u, err := url.Parse(target)
-	if err != nil {
-		path = target
-		ret.Scheme = v1.AddressScheme_IPv4
-	} else {
-		path = u.Path
-	}
-	paths := strings.Split(path, ":")
-	if len(paths) > 0 {
-		if port, err := strconv.ParseInt(paths[1], 10, 32); err != nil {
-			ret.Addresses[0].Port = int32(port)
-		}
-	}
-	ret.Addresses[0].Host = paths[0]
-	return ret
-}
-
-func (ns *defaultNameServer) sync() {
-	f := func() {
-		ns.router.Range(func(k, v interface{}) bool {
-			ctx, _ := context.WithTimeout(context.TODO(), ns.opts.timeout)
-			item, _ := ns.queryRoute(ctx, k.(string))
-			if !reflect.DeepEqual(item, v) {
-				ns.router.Store(k, item)
-				b, ok := ns.brokers.Load(k)
-				if ok {
-					bo, ok := b.(Broker)
-					if ok {
-						bo.SetPartition(item)
-					}
-				}
-			}
-			return true
-		})
-	}
-	ticker.Tick(f, ns.opts.tickerDuration, ns.done)
-}
-
-func (ns *defaultNameServer) GracefulStop() error {
-	close(ns.done)
-	ns.done <- struct{}{}
-	ns.brokers.Range(func(k, v interface{}) bool {
-		broker, ok := v.(Broker)
-		if ok {
-			_ = broker.GracefulStop()
-		}
-		return true
-	})
-	return ns.conn.Close()
-}
diff --git a/golang/producer.go b/golang/producer.go
index e957c2e..aab3e24 100644
--- a/golang/producer.go
+++ b/golang/producer.go
@@ -19,43 +19,240 @@ package golang
 
 import (
 	"context"
+	"errors"
+	"fmt"
+	"log"
+	"sync"
+	"time"
+
+	v2 "github.com/apache/rocketmq-clients/golang/protocol/v2"
+	"github.com/apache/rocketmq-clients/golang/utils"
 )
 
 type Producer interface {
-	Send(context.Context, *Message) (*SendReceipt, error)
+	Send(context.Context, *Message) ([]*SendReceipt, error)
+	SendAsync(context.Context, *Message, func(context.Context, []*SendReceipt, error))
 	GracefulStop() error
 }
 
 type producer struct {
-	po producerOptions
-	ns NameServer
+	po  producerOptions
+	cli *defaultClient
+
+	isolated                       sync.Map
+	publishingRouteDataResultCache sync.Map
 }
 
 var _ = Producer(&producer{})
 
+func (p *producer) takeMessageQueues(plb PublishingLoadBalancer) ([]*v2.MessageQueue, error) {
+	return plb.TakeMessageQueues(p.isolated, p.getRetryMaxAttempts())
+}
+
+func (p *producer) getPublishingTopicRouteResult(ctx context.Context, topic string) (PublishingLoadBalancer, error) {
+	item, ok := p.publishingRouteDataResultCache.Load(topic)
+	if ok {
+		if ret, ok := item.(PublishingLoadBalancer); ok {
+			return ret, nil
+		}
+	}
+	route, err := p.cli.GetMessageQueues(ctx, topic)
+	if err != nil {
+		return nil, err
+	}
+	plb, err := NewPublishingLoadBalancer(route)
+	if err != nil {
+		return nil, err
+	}
+	p.publishingRouteDataResultCache.Store(topic, plb)
+	return plb, nil
+}
+
+func (p *producer) wrapSendMessageRequest(pMsgs []*PublishingMessage) (*v2.SendMessageRequest, error) {
+	smr := &v2.SendMessageRequest{
+		Messages: []*v2.Message{},
+	}
+	for _, pMsg := range pMsgs {
+		msgV2, err := pMsg.toProtobuf()
+		if err != nil {
+			return nil, fmt.Errorf("wrapSendMessageRequest faild, {%v}", err)
+		}
+		smr.Messages = append(smr.Messages, msgV2)
+	}
+	return smr, nil
+}
+
 func NewProducer(config *Config, opts ...ProducerOption) (Producer, error) {
 	po := &defaultProducerOptions
 	for _, opt := range opts {
 		opt.apply(po)
 	}
-	ns, err := po.nameServerFunc(config, WithBrokerOptions(WithProducer()))
+	cli, err := po.clientFunc(config)
 	if err != nil {
 		return nil, err
 	}
+
 	return &producer{
-		po: *po,
-		ns: ns,
+		po:  *po,
+		cli: cli.(*defaultClient),
 	}, nil
 }
+func (p *producer) getRetryMaxAttempts() int {
+	return 3
+}
+func (p *producer) getNextAttemptDelay(nextAttempt int) time.Duration {
+	return time.Second
+}
+
+// TODO refer to java sdk.
+func (p *producer) send1(ctx context.Context, topic string, messageType v2.MessageType,
+	candidates []*v2.MessageQueue, pubMessages []*PublishingMessage, attempt int) ([]*SendReceipt, error) {
+
+	ctx = p.cli.Sign(ctx)
+
+	idx := utils.Mod(int32(attempt)-1, len(candidates))
+	selectMessageQueue := candidates[idx]
+
+	// TODO Determine whether the messageType matches.
+	// producerSettings.isValidateMessageType() && !messageQueue.matchMessageType(messageType)
+
+	endpoints := selectMessageQueue.GetBroker().GetEndpoints()
+	sendReq, err := p.wrapSendMessageRequest(pubMessages)
+	if err != nil {
+		return nil, err
+	}
+	resp, err := p.cli.clientManager.SendMessage(ctx, endpoints, sendReq, time.Second*5)
+	// processSendResponse
+	tooManyRequests := false
+	if err == nil && resp.GetStatus().GetCode() != v2.Code_OK {
+		tooManyRequests = resp.GetStatus().GetCode() == v2.Code_TOO_MANY_REQUESTS
+		err = errors.New(resp.String())
+	}
+	maxAttempts := p.getRetryMaxAttempts()
+	if err != nil {
+		// retry
+		for _, address := range endpoints.GetAddresses() {
+			p.isolated.Store(address.String(), true)
+		}
+		if attempt >= maxAttempts {
+			log.Printf("Failed to send message(s) finally, run out of attempt times, topic=%s, maxAttempts=%d, attempt=%d, endpoints=%s, clientId=%s",
+				topic, maxAttempts, attempt, endpoints.String(), p.cli.clientID)
+			return nil, err
+		}
+		// No need more attempts for transactional message.
+		if messageType == v2.MessageType_TRANSACTION {
+			log.Printf("Failed to send transactional message finally, topic=%s, maxAttempts=%d, attempt=%d, endpoints=%s, clientId=%s",
+				topic, 1, attempt, endpoints.String(), p.cli.clientID)
+			return nil, err
+		}
+		// Try to do more attempts.
+		nextAttempt := attempt + 1
+		// Retry immediately if the request is not throttled.
+		if tooManyRequests {
+			time.Sleep(p.getNextAttemptDelay(nextAttempt))
+		}
+		return p.send1(ctx, topic, messageType, candidates, pubMessages, nextAttempt)
+	}
+
+	var res []*SendReceipt
+	for i := 0; i < len(resp.GetEntries()); i++ {
+		res = append(res, &SendReceipt{
+			MessageID: resp.GetEntries()[i].GetMessageId(),
+		})
+	}
+	if attempt > 1 {
+		log.Printf("Resend message successfully, topic=%s, maxAttempts=%d, attempt=%d, endpoints=%s, clientId=%s",
+			topic, maxAttempts, attempt, endpoints.String(), p.cli.clientID)
+	}
+	return res, nil
+	// sendRequest := b.getSendMessageRequest(msg)
+
+	// b, err := p.ns.GetBroker(ctx, msg.Topic)
+	// if err != nil {
+	// 	return nil, err
+	// }
+	// return b.Send(ctx, msg)
+
+	// return nil, nil
+}
+
+// TODO refer to java sdk.
+func (p *producer) send0(ctx context.Context, msgs []*Message, txEnabled bool) ([]*SendReceipt, error) {
+	// check topic Name
+	topicName := msgs[0].Topic
+	for _, msg := range msgs {
+		if msg.Topic != topicName {
+			return nil, fmt.Errorf("Messages to send have different topics")
+		}
+	}
 
-func (p *producer) Send(ctx context.Context, msg *Message) (*SendReceipt, error) {
-	b, err := p.ns.GetBroker(ctx, msg.Topic)
+	pubMessages := make([]*PublishingMessage, len(msgs))
+	for idx, msg := range msgs {
+		pubMessage, err := NewPublishingMessage(msg, txEnabled)
+		if err != nil {
+			return nil, err
+		}
+		pubMessages[idx] = pubMessage
+	}
+
+	// check topic Name
+	messageType := pubMessages[0].messageType
+	for _, pubMessage := range pubMessages {
+		if pubMessage.messageType != messageType {
+			return nil, fmt.Errorf("Messages to send have different types, please check")
+		}
+	}
+
+	var messageGroup string
+	// Message group must be same if message type is FIFO, or no need to proceed.
+	if messageType == v2.MessageType_FIFO {
+		messageGroup = pubMessages[0].msg.GetMessageGroup()
+		for _, pubMessage := range pubMessages {
+			if pubMessage.msg.GetMessageGroup() != messageGroup {
+				return nil, fmt.Errorf("FIFO messages to send have different message groups")
+			}
+		}
+	}
+
+	pubLoadBalancer, err := p.getPublishingTopicRouteResult(ctx, topicName)
 	if err != nil {
 		return nil, err
 	}
-	return b.Send(ctx, msg)
+	var candidates []*v2.MessageQueue
+	if len(messageGroup) == 0 {
+		candidates, err = p.takeMessageQueues(pubLoadBalancer)
+		for _, v := range candidates {
+			str := v.Broker.Endpoints.String()
+			fmt.Println(str)
+		}
+	} else {
+		candidates, err = pubLoadBalancer.TakeMessageQueueByMessageGroup(messageGroup)
+	}
+	if len(candidates) == 0 {
+		return nil, fmt.Errorf("no broker available to sendMessage")
+	}
+	return p.send1(ctx, topicName, messageType, candidates, pubMessages, 1)
+
+	// broker, err := p.ns.GetBroker(ctx, msgs[0].Topic)
+	// if err != nil {
+	// 	return nil, err
+	// }
+	// return broker.Send(ctx, msgs[0])
+}
+
+func (p *producer) Send(ctx context.Context, msg *Message) ([]*SendReceipt, error) {
+	msgs := []*Message{msg}
+	return p.send0(ctx, msgs, false)
+}
+
+func (p *producer) SendAsync(ctx context.Context, msg *Message, f func(context.Context, []*SendReceipt, error)) {
+	go func() {
+		msgs := []*Message{msg}
+		resp, err := p.send0(ctx, msgs, false)
+		f(ctx, resp, err)
+	}()
 }
 
 func (p *producer) GracefulStop() error {
-	return p.ns.GracefulStop()
+	return p.cli.GracefulStop()
 }
diff --git a/golang/producer_options.go b/golang/producer_options.go
index 218d9ad..66f738f 100644
--- a/golang/producer_options.go
+++ b/golang/producer_options.go
@@ -18,11 +18,11 @@
 package golang
 
 type producerOptions struct {
-	nameServerFunc NewNameServerFunc
+	clientFunc NewClientFunc
 }
 
 var defaultProducerOptions = producerOptions{
-	nameServerFunc: NewNameServer,
+	clientFunc: NewClient,
 }
 
 // A ProducerOption sets options such as tls.Config, etc.
@@ -46,10 +46,10 @@ func newFuncProducerOption(f func(options *producerOptions)) *funcProducerOption
 	}
 }
 
-// WithNameServerFunc returns a ProducerOption that sets NameServerFunc for producer.
+// WithClientFunc returns a ProducerOption that sets ClientFunc for producer.
 // Default is nameserver.New.
-func WithNameServerFunc(f NewNameServerFunc) ProducerOption {
+func WithClientFunc(f NewClientFunc) ProducerOption {
 	return newFuncProducerOption(func(o *producerOptions) {
-		o.nameServerFunc = f
+		o.clientFunc = f
 	})
 }
diff --git a/golang/protocol/v1/admin.pb.go b/golang/protocol/v1/admin.pb.go
deleted file mode 100644
index 1b0a60a..0000000
--- a/golang/protocol/v1/admin.pb.go
+++ /dev/null
@@ -1,246 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: apache/rocketmq/v1/admin.proto
-
-package v1
-
-import (
-	context "context"
-	fmt "fmt"
-	math "math"
-
-	proto "github.com/golang/protobuf/proto"
-	grpc "google.golang.org/grpc"
-	codes "google.golang.org/grpc/codes"
-	status "google.golang.org/grpc/status"
-)
-
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
-
-type ChangeLogLevelRequest_Level int32
-
-const (
-	ChangeLogLevelRequest_TRACE ChangeLogLevelRequest_Level = 0
-	ChangeLogLevelRequest_DEBUG ChangeLogLevelRequest_Level = 1
-	ChangeLogLevelRequest_INFO  ChangeLogLevelRequest_Level = 2
-	ChangeLogLevelRequest_WARN  ChangeLogLevelRequest_Level = 3
-	ChangeLogLevelRequest_ERROR ChangeLogLevelRequest_Level = 4
-)
-
-var ChangeLogLevelRequest_Level_name = map[int32]string{
-	0: "TRACE",
-	1: "DEBUG",
-	2: "INFO",
-	3: "WARN",
-	4: "ERROR",
-}
-
-var ChangeLogLevelRequest_Level_value = map[string]int32{
-	"TRACE": 0,
-	"DEBUG": 1,
-	"INFO":  2,
-	"WARN":  3,
-	"ERROR": 4,
-}
-
-func (x ChangeLogLevelRequest_Level) String() string {
-	return proto.EnumName(ChangeLogLevelRequest_Level_name, int32(x))
-}
-
-func (ChangeLogLevelRequest_Level) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_754a6e0557016a5b, []int{0, 0}
-}
-
-type ChangeLogLevelRequest struct {
-	Level                ChangeLogLevelRequest_Level `protobuf:"varint,1,opt,name=level,proto3,enum=apache.rocketmq.v1.ChangeLogLevelRequest_Level" json:"level,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
-	XXX_unrecognized     []byte                      `json:"-"`
-	XXX_sizecache        int32                       `json:"-"`
-}
-
-func (m *ChangeLogLevelRequest) Reset()         { *m = ChangeLogLevelRequest{} }
-func (m *ChangeLogLevelRequest) String() string { return proto.CompactTextString(m) }
-func (*ChangeLogLevelRequest) ProtoMessage()    {}
-func (*ChangeLogLevelRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_754a6e0557016a5b, []int{0}
-}
-
-func (m *ChangeLogLevelRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ChangeLogLevelRequest.Unmarshal(m, b)
-}
-func (m *ChangeLogLevelRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ChangeLogLevelRequest.Marshal(b, m, deterministic)
-}
-func (m *ChangeLogLevelRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ChangeLogLevelRequest.Merge(m, src)
-}
-func (m *ChangeLogLevelRequest) XXX_Size() int {
-	return xxx_messageInfo_ChangeLogLevelRequest.Size(m)
-}
-func (m *ChangeLogLevelRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ChangeLogLevelRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ChangeLogLevelRequest proto.InternalMessageInfo
-
-func (m *ChangeLogLevelRequest) GetLevel() ChangeLogLevelRequest_Level {
-	if m != nil {
-		return m.Level
-	}
-	return ChangeLogLevelRequest_TRACE
-}
-
-type ChangeLogLevelResponse struct {
-	Remark               string   `protobuf:"bytes,1,opt,name=remark,proto3" json:"remark,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ChangeLogLevelResponse) Reset()         { *m = ChangeLogLevelResponse{} }
-func (m *ChangeLogLevelResponse) String() string { return proto.CompactTextString(m) }
-func (*ChangeLogLevelResponse) ProtoMessage()    {}
-func (*ChangeLogLevelResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_754a6e0557016a5b, []int{1}
-}
-
-func (m *ChangeLogLevelResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ChangeLogLevelResponse.Unmarshal(m, b)
-}
-func (m *ChangeLogLevelResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ChangeLogLevelResponse.Marshal(b, m, deterministic)
-}
-func (m *ChangeLogLevelResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ChangeLogLevelResponse.Merge(m, src)
-}
-func (m *ChangeLogLevelResponse) XXX_Size() int {
-	return xxx_messageInfo_ChangeLogLevelResponse.Size(m)
-}
-func (m *ChangeLogLevelResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ChangeLogLevelResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ChangeLogLevelResponse proto.InternalMessageInfo
-
-func (m *ChangeLogLevelResponse) GetRemark() string {
-	if m != nil {
-		return m.Remark
-	}
-	return ""
-}
-
-func init() {
-	proto.RegisterEnum("apache.rocketmq.v1.ChangeLogLevelRequest_Level", ChangeLogLevelRequest_Level_name, ChangeLogLevelRequest_Level_value)
-	proto.RegisterType((*ChangeLogLevelRequest)(nil), "apache.rocketmq.v1.ChangeLogLevelRequest")
-	proto.RegisterType((*ChangeLogLevelResponse)(nil), "apache.rocketmq.v1.ChangeLogLevelResponse")
-}
-
-func init() { proto.RegisterFile("apache/rocketmq/v1/admin.proto", fileDescriptor_754a6e0557016a5b) }
-
-var fileDescriptor_754a6e0557016a5b = []byte{
-	// 253 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4b, 0x2c, 0x48, 0x4c,
-	0xce, 0x48, 0xd5, 0x2f, 0xca, 0x4f, 0xce, 0x4e, 0x2d, 0xc9, 0x2d, 0xd4, 0x2f, 0x33, 0xd4, 0x4f,
-	0x4c, 0xc9, 0xcd, 0xcc, 0xd3, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x82, 0xc8, 0xeb, 0xc1,
-	0xe4, 0xf5, 0xca, 0x0c, 0x95, 0xe6, 0x30, 0x72, 0x89, 0x3a, 0x67, 0x24, 0xe6, 0xa5, 0xa7, 0xfa,
-	0xe4, 0xa7, 0xfb, 0xa4, 0x96, 0xa5, 0xe6, 0x04, 0xa5, 0x16, 0x96, 0xa6, 0x16, 0x97, 0x08, 0xb9,
-	0x72, 0xb1, 0xe6, 0x80, 0xf8, 0x12, 0x8c, 0x0a, 0x8c, 0x1a, 0x7c, 0x46, 0xfa, 0x7a, 0x98, 0xba,
-	0xf5, 0xb0, 0xea, 0xd4, 0x83, 0x70, 0x20, 0xba, 0x95, 0x6c, 0xb8, 0x58, 0xc1, 0x7c, 0x21, 0x4e,
-	0x2e, 0xd6, 0x90, 0x20, 0x47, 0x67, 0x57, 0x01, 0x06, 0x10, 0xd3, 0xc5, 0xd5, 0x29, 0xd4, 0x5d,
-	0x80, 0x51, 0x88, 0x83, 0x8b, 0xc5, 0xd3, 0xcf, 0xcd, 0x5f, 0x80, 0x09, 0xc4, 0x0a, 0x77, 0x0c,
-	0xf2, 0x13, 0x60, 0x06, 0x49, 0xbb, 0x06, 0x05, 0xf9, 0x07, 0x09, 0xb0, 0x28, 0x19, 0x70, 0x89,
-	0xa1, 0xdb, 0x51, 0x5c, 0x90, 0x9f, 0x57, 0x9c, 0x2a, 0x24, 0xc6, 0xc5, 0x56, 0x94, 0x9a, 0x9b,
-	0x58, 0x94, 0x0d, 0x76, 0x1f, 0x67, 0x10, 0x94, 0x67, 0x54, 0xc4, 0xc5, 0xea, 0x08, 0xf2, 0xb3,
-	0x50, 0x26, 0x17, 0x1f, 0xaa, 0x56, 0x21, 0x4d, 0xa2, 0xbd, 0x20, 0xa5, 0x45, 0x8c, 0x52, 0x88,
-	0x4b, 0x94, 0x18, 0x9c, 0x34, 0xb8, 0xb0, 0x04, 0xad, 0x13, 0xbb, 0x6f, 0x20, 0xd8, 0x25, 0x01,
-	0x8c, 0x0b, 0x18, 0x19, 0x6f, 0x30, 0x32, 0xfe, 0x60, 0x64, 0x4c, 0x62, 0x03, 0xc7, 0x84, 0x31,
-	0x20, 0x00, 0x00, 0xff, 0xff, 0x24, 0x51, 0x46, 0xe5, 0xab, 0x01, 0x00, 0x00,
-}
-
-// Reference imports to suppress errors if they are not otherwise used.
-var _ context.Context
-var _ grpc.ClientConn
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the grpc package it is being compiled against.
-const _ = grpc.SupportPackageIsVersion4
-
-// AdminClient is the client API for Admin service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type AdminClient interface {
-	ChangeLogLevel(ctx context.Context, in *ChangeLogLevelRequest, opts ...grpc.CallOption) (*ChangeLogLevelResponse, error)
-}
-
-type adminClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewAdminClient(cc *grpc.ClientConn) AdminClient {
-	return &adminClient{cc}
-}
-
-func (c *adminClient) ChangeLogLevel(ctx context.Context, in *ChangeLogLevelRequest, opts ...grpc.CallOption) (*ChangeLogLevelResponse, error) {
-	out := new(ChangeLogLevelResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.Admin/ChangeLogLevel", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// AdminServer is the server API for Admin service.
-type AdminServer interface {
-	ChangeLogLevel(context.Context, *ChangeLogLevelRequest) (*ChangeLogLevelResponse, error)
-}
-
-// UnimplementedAdminServer can be embedded to have forward compatible implementations.
-type UnimplementedAdminServer struct {
-}
-
-func (*UnimplementedAdminServer) ChangeLogLevel(ctx context.Context, req *ChangeLogLevelRequest) (*ChangeLogLevelResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ChangeLogLevel not implemented")
-}
-
-func RegisterAdminServer(s *grpc.Server, srv AdminServer) {
-	s.RegisterService(&_Admin_serviceDesc, srv)
-}
-
-func _Admin_ChangeLogLevel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ChangeLogLevelRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(AdminServer).ChangeLogLevel(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.Admin/ChangeLogLevel",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(AdminServer).ChangeLogLevel(ctx, req.(*ChangeLogLevelRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _Admin_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "apache.rocketmq.v1.Admin",
-	HandlerType: (*AdminServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "ChangeLogLevel",
-			Handler:    _Admin_ChangeLogLevel_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "apache/rocketmq/v1/admin.proto",
-}
diff --git a/golang/protocol/v1/definition.pb.go b/golang/protocol/v1/definition.pb.go
deleted file mode 100644
index f32f0e4..0000000
--- a/golang/protocol/v1/definition.pb.go
+++ /dev/null
@@ -1,1407 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: apache/rocketmq/v1/definition.proto
-
-package v1
-
-import (
-	fmt "fmt"
-	math "math"
-
-	proto "github.com/golang/protobuf/proto"
-	durationpb "google.golang.org/protobuf/types/known/durationpb"
-	timestamppb "google.golang.org/protobuf/types/known/timestamppb"
-)
-
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
-
-type Permission int32
-
-const (
-	Permission_NONE       Permission = 0
-	Permission_READ       Permission = 1
-	Permission_WRITE      Permission = 2
-	Permission_READ_WRITE Permission = 3
-)
-
-var Permission_name = map[int32]string{
-	0: "NONE",
-	1: "READ",
-	2: "WRITE",
-	3: "READ_WRITE",
-}
-
-var Permission_value = map[string]int32{
-	"NONE":       0,
-	"READ":       1,
-	"WRITE":      2,
-	"READ_WRITE": 3,
-}
-
-func (x Permission) String() string {
-	return proto.EnumName(Permission_name, int32(x))
-}
-
-func (Permission) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{0}
-}
-
-type FilterType int32
-
-const (
-	FilterType_TAG FilterType = 0
-	FilterType_SQL FilterType = 1
-)
-
-var FilterType_name = map[int32]string{
-	0: "TAG",
-	1: "SQL",
-}
-
-var FilterType_value = map[string]int32{
-	"TAG": 0,
-	"SQL": 1,
-}
-
-func (x FilterType) String() string {
-	return proto.EnumName(FilterType_name, int32(x))
-}
-
-func (FilterType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{1}
-}
-
-type ConsumeModel int32
-
-const (
-	ConsumeModel_CLUSTERING   ConsumeModel = 0
-	ConsumeModel_BROADCASTING ConsumeModel = 1
-)
-
-var ConsumeModel_name = map[int32]string{
-	0: "CLUSTERING",
-	1: "BROADCASTING",
-}
-
-var ConsumeModel_value = map[string]int32{
-	"CLUSTERING":   0,
-	"BROADCASTING": 1,
-}
-
-func (x ConsumeModel) String() string {
-	return proto.EnumName(ConsumeModel_name, int32(x))
-}
-
-func (ConsumeModel) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{2}
-}
-
-type ConsumePolicy int32
-
-const (
-	ConsumePolicy_RESUME           ConsumePolicy = 0
-	ConsumePolicy_PLAYBACK         ConsumePolicy = 1
-	ConsumePolicy_DISCARD          ConsumePolicy = 2
-	ConsumePolicy_TARGET_TIMESTAMP ConsumePolicy = 3
-)
-
-var ConsumePolicy_name = map[int32]string{
-	0: "RESUME",
-	1: "PLAYBACK",
-	2: "DISCARD",
-	3: "TARGET_TIMESTAMP",
-}
-
-var ConsumePolicy_value = map[string]int32{
-	"RESUME":           0,
-	"PLAYBACK":         1,
-	"DISCARD":          2,
-	"TARGET_TIMESTAMP": 3,
-}
-
-func (x ConsumePolicy) String() string {
-	return proto.EnumName(ConsumePolicy_name, int32(x))
-}
-
-func (ConsumePolicy) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{3}
-}
-
-type ConsumeMessageType int32
-
-const (
-	ConsumeMessageType_ACTIVE  ConsumeMessageType = 0
-	ConsumeMessageType_PASSIVE ConsumeMessageType = 1
-)
-
-var ConsumeMessageType_name = map[int32]string{
-	0: "ACTIVE",
-	1: "PASSIVE",
-}
-
-var ConsumeMessageType_value = map[string]int32{
-	"ACTIVE":  0,
-	"PASSIVE": 1,
-}
-
-func (x ConsumeMessageType) String() string {
-	return proto.EnumName(ConsumeMessageType_name, int32(x))
-}
-
-func (ConsumeMessageType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{4}
-}
-
-type AddressScheme int32
-
-const (
-	AddressScheme_IPv4        AddressScheme = 0
-	AddressScheme_IPv6        AddressScheme = 1
-	AddressScheme_DOMAIN_NAME AddressScheme = 2
-)
-
-var AddressScheme_name = map[int32]string{
-	0: "IPv4",
-	1: "IPv6",
-	2: "DOMAIN_NAME",
-}
-
-var AddressScheme_value = map[string]int32{
-	"IPv4":        0,
-	"IPv6":        1,
-	"DOMAIN_NAME": 2,
-}
-
-func (x AddressScheme) String() string {
-	return proto.EnumName(AddressScheme_name, int32(x))
-}
-
-func (AddressScheme) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{5}
-}
-
-type MessageType int32
-
-const (
-	MessageType_NORMAL MessageType = 0
-	// Sequenced message
-	MessageType_FIFO MessageType = 1
-	// Messages that are delivered after the specified duration.
-	MessageType_DELAY MessageType = 2
-	// Messages that are transactional. Only committed messages are delivered to
-	// subscribers.
-	MessageType_TRANSACTION MessageType = 3
-)
-
-var MessageType_name = map[int32]string{
-	0: "NORMAL",
-	1: "FIFO",
-	2: "DELAY",
-	3: "TRANSACTION",
-}
-
-var MessageType_value = map[string]int32{
-	"NORMAL":      0,
-	"FIFO":        1,
-	"DELAY":       2,
-	"TRANSACTION": 3,
-}
-
-func (x MessageType) String() string {
-	return proto.EnumName(MessageType_name, int32(x))
-}
-
-func (MessageType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{6}
-}
-
-type DigestType int32
-
-const (
-	// CRC algorithm achieves goal of detecting random data error with lowest
-	// computation overhead.
-	DigestType_CRC32 DigestType = 0
-	// MD5 algorithm achieves good balance between collision rate and computation
-	// overhead.
-	DigestType_MD5 DigestType = 1
-	// SHA-family has substantially fewer collision with fair amount of
-	// computation.
-	DigestType_SHA1 DigestType = 2
-)
-
-var DigestType_name = map[int32]string{
-	0: "CRC32",
-	1: "MD5",
-	2: "SHA1",
-}
-
-var DigestType_value = map[string]int32{
-	"CRC32": 0,
-	"MD5":   1,
-	"SHA1":  2,
-}
-
-func (x DigestType) String() string {
-	return proto.EnumName(DigestType_name, int32(x))
-}
-
-func (DigestType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{7}
-}
-
-type Encoding int32
-
-const (
-	Encoding_IDENTITY Encoding = 0
-	Encoding_GZIP     Encoding = 1
-)
-
-var Encoding_name = map[int32]string{
-	0: "IDENTITY",
-	1: "GZIP",
-}
-
-var Encoding_value = map[string]int32{
-	"IDENTITY": 0,
-	"GZIP":     1,
-}
-
-func (x Encoding) String() string {
-	return proto.EnumName(Encoding_name, int32(x))
-}
-
-func (Encoding) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{8}
-}
-
-type QueryOffsetPolicy int32
-
-const (
-	// Use this option if client wishes to playback all existing messages.
-	QueryOffsetPolicy_BEGINNING QueryOffsetPolicy = 0
-	// Use this option if client wishes to skip all existing messages.
-	QueryOffsetPolicy_END QueryOffsetPolicy = 1
-	// Use this option if time-based seek is targeted.
-	QueryOffsetPolicy_TIME_POINT QueryOffsetPolicy = 2
-)
-
-var QueryOffsetPolicy_name = map[int32]string{
-	0: "BEGINNING",
-	1: "END",
-	2: "TIME_POINT",
-}
-
-var QueryOffsetPolicy_value = map[string]int32{
-	"BEGINNING":  0,
-	"END":        1,
-	"TIME_POINT": 2,
-}
-
-func (x QueryOffsetPolicy) String() string {
-	return proto.EnumName(QueryOffsetPolicy_name, int32(x))
-}
-
-func (QueryOffsetPolicy) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{9}
-}
-
-type FilterExpression struct {
-	Type                 FilterType `protobuf:"varint,1,opt,name=type,proto3,enum=apache.rocketmq.v1.FilterType" json:"type,omitempty"`
-	Expression           string     `protobuf:"bytes,2,opt,name=expression,proto3" json:"expression,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *FilterExpression) Reset()         { *m = FilterExpression{} }
-func (m *FilterExpression) String() string { return proto.CompactTextString(m) }
-func (*FilterExpression) ProtoMessage()    {}
-func (*FilterExpression) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{0}
-}
-
-func (m *FilterExpression) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FilterExpression.Unmarshal(m, b)
-}
-func (m *FilterExpression) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FilterExpression.Marshal(b, m, deterministic)
-}
-func (m *FilterExpression) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FilterExpression.Merge(m, src)
-}
-func (m *FilterExpression) XXX_Size() int {
-	return xxx_messageInfo_FilterExpression.Size(m)
-}
-func (m *FilterExpression) XXX_DiscardUnknown() {
-	xxx_messageInfo_FilterExpression.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FilterExpression proto.InternalMessageInfo
-
-func (m *FilterExpression) GetType() FilterType {
-	if m != nil {
-		return m.Type
-	}
-	return FilterType_TAG
-}
-
-func (m *FilterExpression) GetExpression() string {
-	if m != nil {
-		return m.Expression
-	}
-	return ""
-}
-
-// Dead lettering is done on a best effort basis. The same message might be
-// dead lettered multiple times.
-//
-// If validation on any of the fields fails at subscription creation/update,
-// the create/update subscription request will fail.
-type DeadLetterPolicy struct {
-	// The maximum number of delivery attempts for any message.
-	//
-	// This field will be honored on a best effort basis.
-	//
-	// If this parameter is 0, a default value of 16 is used.
-	MaxDeliveryAttempts  int32    `protobuf:"varint,1,opt,name=max_delivery_attempts,json=maxDeliveryAttempts,proto3" json:"max_delivery_attempts,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *DeadLetterPolicy) Reset()         { *m = DeadLetterPolicy{} }
-func (m *DeadLetterPolicy) String() string { return proto.CompactTextString(m) }
-func (*DeadLetterPolicy) ProtoMessage()    {}
-func (*DeadLetterPolicy) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{1}
-}
-
-func (m *DeadLetterPolicy) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeadLetterPolicy.Unmarshal(m, b)
-}
-func (m *DeadLetterPolicy) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeadLetterPolicy.Marshal(b, m, deterministic)
-}
-func (m *DeadLetterPolicy) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeadLetterPolicy.Merge(m, src)
-}
-func (m *DeadLetterPolicy) XXX_Size() int {
-	return xxx_messageInfo_DeadLetterPolicy.Size(m)
-}
-func (m *DeadLetterPolicy) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeadLetterPolicy.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeadLetterPolicy proto.InternalMessageInfo
-
-func (m *DeadLetterPolicy) GetMaxDeliveryAttempts() int32 {
-	if m != nil {
-		return m.MaxDeliveryAttempts
-	}
-	return 0
-}
-
-type Resource struct {
-	ResourceNamespace string `protobuf:"bytes,1,opt,name=resource_namespace,json=resourceNamespace,proto3" json:"resource_namespace,omitempty"`
-	// Resource name identifier, which remains unique within the abstract resource
-	// namespace.
-	Name                 string   `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Resource) Reset()         { *m = Resource{} }
-func (m *Resource) String() string { return proto.CompactTextString(m) }
-func (*Resource) ProtoMessage()    {}
-func (*Resource) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{2}
-}
-
-func (m *Resource) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Resource.Unmarshal(m, b)
-}
-func (m *Resource) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Resource.Marshal(b, m, deterministic)
-}
-func (m *Resource) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Resource.Merge(m, src)
-}
-func (m *Resource) XXX_Size() int {
-	return xxx_messageInfo_Resource.Size(m)
-}
-func (m *Resource) XXX_DiscardUnknown() {
-	xxx_messageInfo_Resource.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Resource proto.InternalMessageInfo
-
-func (m *Resource) GetResourceNamespace() string {
-	if m != nil {
-		return m.ResourceNamespace
-	}
-	return ""
-}
-
-func (m *Resource) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-type ProducerData struct {
-	Group                *Resource `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *ProducerData) Reset()         { *m = ProducerData{} }
-func (m *ProducerData) String() string { return proto.CompactTextString(m) }
-func (*ProducerData) ProtoMessage()    {}
-func (*ProducerData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{3}
-}
-
-func (m *ProducerData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProducerData.Unmarshal(m, b)
-}
-func (m *ProducerData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProducerData.Marshal(b, m, deterministic)
-}
-func (m *ProducerData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProducerData.Merge(m, src)
-}
-func (m *ProducerData) XXX_Size() int {
-	return xxx_messageInfo_ProducerData.Size(m)
-}
-func (m *ProducerData) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProducerData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProducerData proto.InternalMessageInfo
-
-func (m *ProducerData) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-type ConsumerData struct {
-	Group                *Resource            `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	Subscriptions        []*SubscriptionEntry `protobuf:"bytes,2,rep,name=subscriptions,proto3" json:"subscriptions,omitempty"`
-	ConsumeModel         ConsumeModel         `protobuf:"varint,3,opt,name=consume_model,json=consumeModel,proto3,enum=apache.rocketmq.v1.ConsumeModel" json:"consume_model,omitempty"`
-	ConsumePolicy        ConsumePolicy        `protobuf:"varint,4,opt,name=consume_policy,json=consumePolicy,proto3,enum=apache.rocketmq.v1.ConsumePolicy" json:"consume_policy,omitempty"`
-	DeadLetterPolicy     *DeadLetterPolicy    `protobuf:"bytes,5,opt,name=dead_letter_policy,json=deadLetterPolicy,proto3" json:"dead_letter_policy,omitempty"`
-	ConsumeType          ConsumeMessageType   `protobuf:"varint,6,opt,name=consume_type,json=consumeType,proto3,enum=apache.rocketmq.v1.ConsumeMessageType" json:"consume_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *ConsumerData) Reset()         { *m = ConsumerData{} }
-func (m *ConsumerData) String() string { return proto.CompactTextString(m) }
-func (*ConsumerData) ProtoMessage()    {}
-func (*ConsumerData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{4}
-}
-
-func (m *ConsumerData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ConsumerData.Unmarshal(m, b)
-}
-func (m *ConsumerData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ConsumerData.Marshal(b, m, deterministic)
-}
-func (m *ConsumerData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ConsumerData.Merge(m, src)
-}
-func (m *ConsumerData) XXX_Size() int {
-	return xxx_messageInfo_ConsumerData.Size(m)
-}
-func (m *ConsumerData) XXX_DiscardUnknown() {
-	xxx_messageInfo_ConsumerData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ConsumerData proto.InternalMessageInfo
-
-func (m *ConsumerData) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *ConsumerData) GetSubscriptions() []*SubscriptionEntry {
-	if m != nil {
-		return m.Subscriptions
-	}
-	return nil
-}
-
-func (m *ConsumerData) GetConsumeModel() ConsumeModel {
-	if m != nil {
-		return m.ConsumeModel
-	}
-	return ConsumeModel_CLUSTERING
-}
-
-func (m *ConsumerData) GetConsumePolicy() ConsumePolicy {
-	if m != nil {
-		return m.ConsumePolicy
-	}
-	return ConsumePolicy_RESUME
-}
-
-func (m *ConsumerData) GetDeadLetterPolicy() *DeadLetterPolicy {
-	if m != nil {
-		return m.DeadLetterPolicy
-	}
-	return nil
-}
-
-func (m *ConsumerData) GetConsumeType() ConsumeMessageType {
-	if m != nil {
-		return m.ConsumeType
-	}
-	return ConsumeMessageType_ACTIVE
-}
-
-type SubscriptionEntry struct {
-	Topic                *Resource         `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
-	Expression           *FilterExpression `protobuf:"bytes,2,opt,name=expression,proto3" json:"expression,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *SubscriptionEntry) Reset()         { *m = SubscriptionEntry{} }
-func (m *SubscriptionEntry) String() string { return proto.CompactTextString(m) }
-func (*SubscriptionEntry) ProtoMessage()    {}
-func (*SubscriptionEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{5}
-}
-
-func (m *SubscriptionEntry) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SubscriptionEntry.Unmarshal(m, b)
-}
-func (m *SubscriptionEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SubscriptionEntry.Marshal(b, m, deterministic)
-}
-func (m *SubscriptionEntry) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SubscriptionEntry.Merge(m, src)
-}
-func (m *SubscriptionEntry) XXX_Size() int {
-	return xxx_messageInfo_SubscriptionEntry.Size(m)
-}
-func (m *SubscriptionEntry) XXX_DiscardUnknown() {
-	xxx_messageInfo_SubscriptionEntry.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SubscriptionEntry proto.InternalMessageInfo
-
-func (m *SubscriptionEntry) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *SubscriptionEntry) GetExpression() *FilterExpression {
-	if m != nil {
-		return m.Expression
-	}
-	return nil
-}
-
-type Address struct {
-	Host                 string   `protobuf:"bytes,1,opt,name=host,proto3" json:"host,omitempty"`
-	Port                 int32    `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Address) Reset()         { *m = Address{} }
-func (m *Address) String() string { return proto.CompactTextString(m) }
-func (*Address) ProtoMessage()    {}
-func (*Address) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{6}
-}
-
-func (m *Address) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Address.Unmarshal(m, b)
-}
-func (m *Address) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Address.Marshal(b, m, deterministic)
-}
-func (m *Address) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Address.Merge(m, src)
-}
-func (m *Address) XXX_Size() int {
-	return xxx_messageInfo_Address.Size(m)
-}
-func (m *Address) XXX_DiscardUnknown() {
-	xxx_messageInfo_Address.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Address proto.InternalMessageInfo
-
-func (m *Address) GetHost() string {
-	if m != nil {
-		return m.Host
-	}
-	return ""
-}
-
-func (m *Address) GetPort() int32 {
-	if m != nil {
-		return m.Port
-	}
-	return 0
-}
-
-type Endpoints struct {
-	Scheme               AddressScheme `protobuf:"varint,1,opt,name=scheme,proto3,enum=apache.rocketmq.v1.AddressScheme" json:"scheme,omitempty"`
-	Addresses            []*Address    `protobuf:"bytes,2,rep,name=addresses,proto3" json:"addresses,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *Endpoints) Reset()         { *m = Endpoints{} }
-func (m *Endpoints) String() string { return proto.CompactTextString(m) }
-func (*Endpoints) ProtoMessage()    {}
-func (*Endpoints) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{7}
-}
-
-func (m *Endpoints) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Endpoints.Unmarshal(m, b)
-}
-func (m *Endpoints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Endpoints.Marshal(b, m, deterministic)
-}
-func (m *Endpoints) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Endpoints.Merge(m, src)
-}
-func (m *Endpoints) XXX_Size() int {
-	return xxx_messageInfo_Endpoints.Size(m)
-}
-func (m *Endpoints) XXX_DiscardUnknown() {
-	xxx_messageInfo_Endpoints.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Endpoints proto.InternalMessageInfo
-
-func (m *Endpoints) GetScheme() AddressScheme {
-	if m != nil {
-		return m.Scheme
-	}
-	return AddressScheme_IPv4
-}
-
-func (m *Endpoints) GetAddresses() []*Address {
-	if m != nil {
-		return m.Addresses
-	}
-	return nil
-}
-
-type Broker struct {
-	// Name of the broker
-	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	// Broker index. Canonically, index = 0 implies that the broker is playing
-	// leader role while brokers with index > 0 play follower role.
-	Id int32 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
-	// Address of the broker, complying with the following scheme
-	// 1. dns:[//authority/]host[:port]
-	// 2. ipv4:address[:port][,address[:port],...] – IPv4 addresses
-	// 3. ipv6:address[:port][,address[:port],...] – IPv6 addresses
-	Endpoints            *Endpoints `protobuf:"bytes,3,opt,name=endpoints,proto3" json:"endpoints,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Broker) Reset()         { *m = Broker{} }
-func (m *Broker) String() string { return proto.CompactTextString(m) }
-func (*Broker) ProtoMessage()    {}
-func (*Broker) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{8}
-}
-
-func (m *Broker) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Broker.Unmarshal(m, b)
-}
-func (m *Broker) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Broker.Marshal(b, m, deterministic)
-}
-func (m *Broker) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Broker.Merge(m, src)
-}
-func (m *Broker) XXX_Size() int {
-	return xxx_messageInfo_Broker.Size(m)
-}
-func (m *Broker) XXX_DiscardUnknown() {
-	xxx_messageInfo_Broker.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Broker proto.InternalMessageInfo
-
-func (m *Broker) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *Broker) GetId() int32 {
-	if m != nil {
-		return m.Id
-	}
-	return 0
-}
-
-func (m *Broker) GetEndpoints() *Endpoints {
-	if m != nil {
-		return m.Endpoints
-	}
-	return nil
-}
-
-type Partition struct {
-	Topic                *Resource  `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
-	Id                   int32      `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
-	Permission           Permission `protobuf:"varint,3,opt,name=permission,proto3,enum=apache.rocketmq.v1.Permission" json:"permission,omitempty"`
-	Broker               *Broker    `protobuf:"bytes,4,opt,name=broker,proto3" json:"broker,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Partition) Reset()         { *m = Partition{} }
-func (m *Partition) String() string { return proto.CompactTextString(m) }
-func (*Partition) ProtoMessage()    {}
-func (*Partition) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{9}
-}
-
-func (m *Partition) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Partition.Unmarshal(m, b)
-}
-func (m *Partition) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Partition.Marshal(b, m, deterministic)
-}
-func (m *Partition) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Partition.Merge(m, src)
-}
-func (m *Partition) XXX_Size() int {
-	return xxx_messageInfo_Partition.Size(m)
-}
-func (m *Partition) XXX_DiscardUnknown() {
-	xxx_messageInfo_Partition.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Partition proto.InternalMessageInfo
-
-func (m *Partition) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *Partition) GetId() int32 {
-	if m != nil {
-		return m.Id
-	}
-	return 0
-}
-
-func (m *Partition) GetPermission() Permission {
-	if m != nil {
-		return m.Permission
-	}
-	return Permission_NONE
-}
-
-func (m *Partition) GetBroker() *Broker {
-	if m != nil {
-		return m.Broker
-	}
-	return nil
-}
-
-// When publishing messages to or subscribing messages from brokers, clients
-// shall include or validate digests of message body to ensure data integrity.
-//
-// For message publishment, when an invalid digest were detected, brokers need
-// respond client with BAD_REQUEST.
-//
-// For messags subscription, when an invalid digest were detected, consumers
-// need to handle this case according to message type:
-// 1) Standard messages should be negatively acknowledged instantly, causing
-// immediate re-delivery; 2) FIFO messages require special RPC, to re-fetch
-// previously acquired messages batch;
-//
-// Message consumption model also affects how invalid digest are handled. When
-// messages are consumed in broadcasting way,
-// TODO: define semantics of invalid-digest-when-broadcasting.
-type Digest struct {
-	Type                 DigestType `protobuf:"varint,1,opt,name=type,proto3,enum=apache.rocketmq.v1.DigestType" json:"type,omitempty"`
-	Checksum             string     `protobuf:"bytes,2,opt,name=checksum,proto3" json:"checksum,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Digest) Reset()         { *m = Digest{} }
-func (m *Digest) String() string { return proto.CompactTextString(m) }
-func (*Digest) ProtoMessage()    {}
-func (*Digest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{10}
-}
-
-func (m *Digest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Digest.Unmarshal(m, b)
-}
-func (m *Digest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Digest.Marshal(b, m, deterministic)
-}
-func (m *Digest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Digest.Merge(m, src)
-}
-func (m *Digest) XXX_Size() int {
-	return xxx_messageInfo_Digest.Size(m)
-}
-func (m *Digest) XXX_DiscardUnknown() {
-	xxx_messageInfo_Digest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Digest proto.InternalMessageInfo
-
-func (m *Digest) GetType() DigestType {
-	if m != nil {
-		return m.Type
-	}
-	return DigestType_CRC32
-}
-
-func (m *Digest) GetChecksum() string {
-	if m != nil {
-		return m.Checksum
-	}
-	return ""
-}
-
-type SystemAttribute struct {
-	// Tag
-	Tag string `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"`
-	// Message keys
-	Keys []string `protobuf:"bytes,2,rep,name=keys,proto3" json:"keys,omitempty"`
-	// Message identifier, client-side generated, remains unique.
-	// if message_id is empty, the send message request will be aborted with
-	// status `INVALID_ARGUMENT`
-	MessageId string `protobuf:"bytes,3,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	// Message body digest
-	BodyDigest *Digest `protobuf:"bytes,4,opt,name=body_digest,json=bodyDigest,proto3" json:"body_digest,omitempty"`
-	// Message body encoding. Candidate options are identity, gzip, snappy etc.
-	BodyEncoding Encoding `protobuf:"varint,5,opt,name=body_encoding,json=bodyEncoding,proto3,enum=apache.rocketmq.v1.Encoding" json:"body_encoding,omitempty"`
-	// Message type, normal, FIFO or transactional.
-	MessageType MessageType `protobuf:"varint,6,opt,name=message_type,json=messageType,proto3,enum=apache.rocketmq.v1.MessageType" json:"message_type,omitempty"`
-	// Message born time-point.
-	BornTimestamp *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=born_timestamp,json=bornTimestamp,proto3" json:"born_timestamp,omitempty"`
-	// Message born host. Valid options are IPv4, IPv6 or client host domain name.
-	BornHost string `protobuf:"bytes,8,opt,name=born_host,json=bornHost,proto3" json:"born_host,omitempty"`
-	// Time-point at which the message is stored in the broker.
-	StoreTimestamp *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=store_timestamp,json=storeTimestamp,proto3" json:"store_timestamp,omitempty"`
-	// The broker that stores this message. It may be name, IP or arbitrary
-	// identifier that uniquely identify the broker.
-	StoreHost string `protobuf:"bytes,10,opt,name=store_host,json=storeHost,proto3" json:"store_host,omitempty"`
-	// Types that are valid to be assigned to TimedDelivery:
-	//	*SystemAttribute_DeliveryTimestamp
-	//	*SystemAttribute_DelayLevel
-	TimedDelivery isSystemAttribute_TimedDelivery `protobuf_oneof:"timed_delivery"`
-	// If a message is acquired by way of POP, this field holds the receipt.
-	// Clients use the receipt to acknowledge or negatively acknowledge the
-	// message.
-	ReceiptHandle string `protobuf:"bytes,13,opt,name=receipt_handle,json=receiptHandle,proto3" json:"receipt_handle,omitempty"`
-	// Partition identifier in which a message is physically stored.
-	PartitionId int32 `protobuf:"varint,14,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
-	// Partition offset at which a message is stored.
-	PartitionOffset int64 `protobuf:"varint,15,opt,name=partition_offset,json=partitionOffset,proto3" json:"partition_offset,omitempty"`
-	// Period of time servers would remain invisible once a message is acquired.
-	InvisiblePeriod *durationpb.Duration `protobuf:"bytes,16,opt,name=invisible_period,json=invisiblePeriod,proto3" json:"invisible_period,omitempty"`
-	// Business code may failed to process messages for the moment. Hence, clients
-	// may request servers to deliver them again using certain back-off strategy,
-	// the attempt is 1 not 0 if message is delivered first time.
-	DeliveryAttempt int32 `protobuf:"varint,17,opt,name=delivery_attempt,json=deliveryAttempt,proto3" json:"delivery_attempt,omitempty"`
-	// Message producer load-balance group if applicable.
-	ProducerGroup *Resource `protobuf:"bytes,18,opt,name=producer_group,json=producerGroup,proto3" json:"producer_group,omitempty"`
-	MessageGroup  string    `protobuf:"bytes,19,opt,name=message_group,json=messageGroup,proto3" json:"message_group,omitempty"`
-	// Trace context.
-	TraceContext string `protobuf:"bytes,20,opt,name=trace_context,json=traceContext,proto3" json:"trace_context,omitempty"`
-	// Delay time of first recover orphaned transaction request from server.
-	OrphanedTransactionRecoveryPeriod *durationpb.Duration `protobuf:"bytes,21,opt,name=orphaned_transaction_recovery_period,json=orphanedTransactionRecoveryPeriod,proto3" json:"orphaned_transaction_recovery_period,omitempty"`
-	XXX_NoUnkeyedLiteral              struct{}             `json:"-"`
-	XXX_unrecognized                  []byte               `json:"-"`
-	XXX_sizecache                     int32                `json:"-"`
-}
-
-func (m *SystemAttribute) Reset()         { *m = SystemAttribute{} }
-func (m *SystemAttribute) String() string { return proto.CompactTextString(m) }
-func (*SystemAttribute) ProtoMessage()    {}
-func (*SystemAttribute) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{11}
-}
-
-func (m *SystemAttribute) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SystemAttribute.Unmarshal(m, b)
-}
-func (m *SystemAttribute) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SystemAttribute.Marshal(b, m, deterministic)
-}
-func (m *SystemAttribute) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SystemAttribute.Merge(m, src)
-}
-func (m *SystemAttribute) XXX_Size() int {
-	return xxx_messageInfo_SystemAttribute.Size(m)
-}
-func (m *SystemAttribute) XXX_DiscardUnknown() {
-	xxx_messageInfo_SystemAttribute.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SystemAttribute proto.InternalMessageInfo
-
-func (m *SystemAttribute) GetTag() string {
-	if m != nil {
-		return m.Tag
-	}
-	return ""
-}
-
-func (m *SystemAttribute) GetKeys() []string {
-	if m != nil {
-		return m.Keys
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
-	}
-	return ""
-}
-
-func (m *SystemAttribute) GetBodyDigest() *Digest {
-	if m != nil {
-		return m.BodyDigest
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetBodyEncoding() Encoding {
-	if m != nil {
-		return m.BodyEncoding
-	}
-	return Encoding_IDENTITY
-}
-
-func (m *SystemAttribute) GetMessageType() MessageType {
-	if m != nil {
-		return m.MessageType
-	}
-	return MessageType_NORMAL
-}
-
-func (m *SystemAttribute) GetBornTimestamp() *timestamppb.Timestamp {
-	if m != nil {
-		return m.BornTimestamp
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetBornHost() string {
-	if m != nil {
-		return m.BornHost
-	}
-	return ""
-}
-
-func (m *SystemAttribute) GetStoreTimestamp() *timestamppb.Timestamp {
-	if m != nil {
-		return m.StoreTimestamp
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetStoreHost() string {
-	if m != nil {
-		return m.StoreHost
-	}
-	return ""
-}
-
-type isSystemAttribute_TimedDelivery interface {
-	isSystemAttribute_TimedDelivery()
-}
-
-type SystemAttribute_DeliveryTimestamp struct {
-	DeliveryTimestamp *timestamppb.Timestamp `protobuf:"bytes,11,opt,name=delivery_timestamp,json=deliveryTimestamp,proto3,oneof"`
-}
-
-type SystemAttribute_DelayLevel struct {
-	DelayLevel int32 `protobuf:"varint,12,opt,name=delay_level,json=delayLevel,proto3,oneof"`
-}
-
-func (*SystemAttribute_DeliveryTimestamp) isSystemAttribute_TimedDelivery() {}
-
-func (*SystemAttribute_DelayLevel) isSystemAttribute_TimedDelivery() {}
-
-func (m *SystemAttribute) GetTimedDelivery() isSystemAttribute_TimedDelivery {
-	if m != nil {
-		return m.TimedDelivery
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetDeliveryTimestamp() *timestamppb.Timestamp {
-	if x, ok := m.GetTimedDelivery().(*SystemAttribute_DeliveryTimestamp); ok {
-		return x.DeliveryTimestamp
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetDelayLevel() int32 {
-	if x, ok := m.GetTimedDelivery().(*SystemAttribute_DelayLevel); ok {
-		return x.DelayLevel
-	}
-	return 0
-}
-
-func (m *SystemAttribute) GetReceiptHandle() string {
-	if m != nil {
-		return m.ReceiptHandle
-	}
-	return ""
-}
-
-func (m *SystemAttribute) GetPartitionId() int32 {
-	if m != nil {
-		return m.PartitionId
-	}
-	return 0
-}
-
-func (m *SystemAttribute) GetPartitionOffset() int64 {
-	if m != nil {
-		return m.PartitionOffset
-	}
-	return 0
-}
-
-func (m *SystemAttribute) GetInvisiblePeriod() *durationpb.Duration {
-	if m != nil {
-		return m.InvisiblePeriod
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetDeliveryAttempt() int32 {
-	if m != nil {
-		return m.DeliveryAttempt
-	}
-	return 0
-}
-
-func (m *SystemAttribute) GetProducerGroup() *Resource {
-	if m != nil {
-		return m.ProducerGroup
-	}
-	return nil
-}
-
-func (m *SystemAttribute) GetMessageGroup() string {
-	if m != nil {
-		return m.MessageGroup
-	}
-	return ""
-}
-
-func (m *SystemAttribute) GetTraceContext() string {
-	if m != nil {
-		return m.TraceContext
-	}
-	return ""
-}
-
-func (m *SystemAttribute) GetOrphanedTransactionRecoveryPeriod() *durationpb.Duration {
-	if m != nil {
-		return m.OrphanedTransactionRecoveryPeriod
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*SystemAttribute) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*SystemAttribute_DeliveryTimestamp)(nil),
-		(*SystemAttribute_DelayLevel)(nil),
-	}
-}
-
-type Message struct {
-	Topic *Resource `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
-	// User defined key-value pairs.
-	// If user_attribute contains the reserved keys by RocketMQ,
-	// the send message request will be aborted with status `INVALID_ARGUMENT`.
-	// See below links for the reserved keys
-	// https://github.com/apache/rocketmq/blob/master/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java#L58
-	UserAttribute        map[string]string `protobuf:"bytes,2,rep,name=user_attribute,json=userAttribute,proto3" json:"user_attribute,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	SystemAttribute      *SystemAttribute  `protobuf:"bytes,3,opt,name=system_attribute,json=systemAttribute,proto3" json:"system_attribute,omitempty"`
-	Body                 []byte            `protobuf:"bytes,4,opt,name=body,proto3" json:"body,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
-}
-
-func (m *Message) Reset()         { *m = Message{} }
-func (m *Message) String() string { return proto.CompactTextString(m) }
-func (*Message) ProtoMessage()    {}
-func (*Message) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{12}
-}
-
-func (m *Message) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Message.Unmarshal(m, b)
-}
-func (m *Message) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Message.Marshal(b, m, deterministic)
-}
-func (m *Message) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Message.Merge(m, src)
-}
-func (m *Message) XXX_Size() int {
-	return xxx_messageInfo_Message.Size(m)
-}
-func (m *Message) XXX_DiscardUnknown() {
-	xxx_messageInfo_Message.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Message proto.InternalMessageInfo
-
-func (m *Message) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *Message) GetUserAttribute() map[string]string {
-	if m != nil {
-		return m.UserAttribute
-	}
-	return nil
-}
-
-func (m *Message) GetSystemAttribute() *SystemAttribute {
-	if m != nil {
-		return m.SystemAttribute
-	}
-	return nil
-}
-
-func (m *Message) GetBody() []byte {
-	if m != nil {
-		return m.Body
-	}
-	return nil
-}
-
-type Assignment struct {
-	Partition            *Partition `protobuf:"bytes,1,opt,name=Partition,proto3" json:"Partition,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Assignment) Reset()         { *m = Assignment{} }
-func (m *Assignment) String() string { return proto.CompactTextString(m) }
-func (*Assignment) ProtoMessage()    {}
-func (*Assignment) Descriptor() ([]byte, []int) {
-	return fileDescriptor_e88b04e3f5e433a6, []int{13}
-}
-
-func (m *Assignment) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Assignment.Unmarshal(m, b)
-}
-func (m *Assignment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Assignment.Marshal(b, m, deterministic)
-}
-func (m *Assignment) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Assignment.Merge(m, src)
-}
-func (m *Assignment) XXX_Size() int {
-	return xxx_messageInfo_Assignment.Size(m)
-}
-func (m *Assignment) XXX_DiscardUnknown() {
-	xxx_messageInfo_Assignment.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Assignment proto.InternalMessageInfo
-
-func (m *Assignment) GetPartition() *Partition {
-	if m != nil {
-		return m.Partition
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("apache.rocketmq.v1.Permission", Permission_name, Permission_value)
-	proto.RegisterEnum("apache.rocketmq.v1.FilterType", FilterType_name, FilterType_value)
-	proto.RegisterEnum("apache.rocketmq.v1.ConsumeModel", ConsumeModel_name, ConsumeModel_value)
-	proto.RegisterEnum("apache.rocketmq.v1.ConsumePolicy", ConsumePolicy_name, ConsumePolicy_value)
-	proto.RegisterEnum("apache.rocketmq.v1.ConsumeMessageType", ConsumeMessageType_name, ConsumeMessageType_value)
-	proto.RegisterEnum("apache.rocketmq.v1.AddressScheme", AddressScheme_name, AddressScheme_value)
-	proto.RegisterEnum("apache.rocketmq.v1.MessageType", MessageType_name, MessageType_value)
-	proto.RegisterEnum("apache.rocketmq.v1.DigestType", DigestType_name, DigestType_value)
-	proto.RegisterEnum("apache.rocketmq.v1.Encoding", Encoding_name, Encoding_value)
-	proto.RegisterEnum("apache.rocketmq.v1.QueryOffsetPolicy", QueryOffsetPolicy_name, QueryOffsetPolicy_value)
-	proto.RegisterType((*FilterExpression)(nil), "apache.rocketmq.v1.FilterExpression")
-	proto.RegisterType((*DeadLetterPolicy)(nil), "apache.rocketmq.v1.DeadLetterPolicy")
-	proto.RegisterType((*Resource)(nil), "apache.rocketmq.v1.Resource")
-	proto.RegisterType((*ProducerData)(nil), "apache.rocketmq.v1.ProducerData")
-	proto.RegisterType((*ConsumerData)(nil), "apache.rocketmq.v1.ConsumerData")
-	proto.RegisterType((*SubscriptionEntry)(nil), "apache.rocketmq.v1.SubscriptionEntry")
-	proto.RegisterType((*Address)(nil), "apache.rocketmq.v1.Address")
-	proto.RegisterType((*Endpoints)(nil), "apache.rocketmq.v1.Endpoints")
-	proto.RegisterType((*Broker)(nil), "apache.rocketmq.v1.Broker")
-	proto.RegisterType((*Partition)(nil), "apache.rocketmq.v1.Partition")
-	proto.RegisterType((*Digest)(nil), "apache.rocketmq.v1.Digest")
-	proto.RegisterType((*SystemAttribute)(nil), "apache.rocketmq.v1.SystemAttribute")
-	proto.RegisterType((*Message)(nil), "apache.rocketmq.v1.Message")
-	proto.RegisterMapType((map[string]string)(nil), "apache.rocketmq.v1.Message.UserAttributeEntry")
-	proto.RegisterType((*Assignment)(nil), "apache.rocketmq.v1.Assignment")
-}
-
-func init() {
-	proto.RegisterFile("apache/rocketmq/v1/definition.proto", fileDescriptor_e88b04e3f5e433a6)
-}
-
-var fileDescriptor_e88b04e3f5e433a6 = []byte{
-	// 1622 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x57, 0xdb, 0x6e, 0xdb, 0xc8,
-	0x19, 0x0e, 0x25, 0xd9, 0x96, 0x7e, 0x1d, 0x4c, 0xcf, 0x66, 0x0b, 0xd5, 0xdb, 0x64, 0x1d, 0x25,
-	0x5b, 0xb8, 0x01, 0x2a, 0x37, 0x4a, 0xbb, 0xed, 0xee, 0x02, 0x45, 0x28, 0x91, 0xb1, 0xe9, 0xe8,
-	0x94, 0x11, 0xd3, 0x22, 0xbd, 0x28, 0x41, 0x91, 0x13, 0x99, 0xb5, 0x78, 0x28, 0x39, 0x12, 0xac,
-	0x27, 0xe8, 0x03, 0xf4, 0x05, 0x7a, 0xdd, 0xa7, 0xe8, 0x4d, 0x51, 0xf4, 0x0d, 0xfa, 0x38, 0xc5,
-	0x1c, 0x48, 0xc9, 0xb2, 0xd6, 0x09, 0x72, 0x37, 0xf3, 0xf3, 0xfb, 0x0f, 0xf3, 0x9f, 0x09, 0x4f,
-	0x9d, 0xd8, 0x71, 0xaf, 0xc8, 0x59, 0x12, 0xb9, 0xd7, 0x84, 0x06, 0x7f, 0x3d, 0x5b, 0xbe, 0x38,
-	0xf3, 0xc8, 0x07, 0x3f, 0xf4, 0xa9, 0x1f, 0x85, 0xed, 0x38, 0x89, 0x68, 0x84, 0x90, 0x00, 0xb5,
-	0x33, 0x50, 0x7b, 0xf9, 0xe2, 0xf8, 0xeb, 0x59, 0x14, 0xcd, 0xe6, 0xe4, 0x8c, 0x23, 0xa6, 0x8b,
-	0x0f, 0x67, 0xd4, 0x0f, 0x48, 0x4a, 0x9d, 0x20, 0x16, 0x4c, 0xc7, 0x8f, 0xb7, 0x01, 0xde, 0x22,
-	0x71, 0xd6, 0x42, 0x5b, 0x73, 0x50, 0x5f, 0xfb, 0x73, 0x4a, 0x12, 0xe3, 0x26, 0x4e, 0x48, 0x9a,
-	0xfa, 0x51, 0x88, 0x3a, 0x50, 0xa2, 0xab, 0x98, 0x34, 0x95, 0x13, 0xe5, 0xb4, 0xd1, 0x79, 0xdc,
-	0xbe, 0xab, 0xb7, 0x2d, 0x78, 0xac, 0x55, 0x4c, 0x30, 0xc7, 0xa2, 0xc7, 0x00, 0x24, 0x97, 0xd0,
-	0x2c, 0x9c, 0x28, 0xa7, 0x15, 0xbc, 0x41, 0xb9, 0x2c, 0x95, 0x8b, 0xaa, 0xd6, 0xea, 0x83, 0xaa,
-	0x13, 0xc7, 0xeb, 0x13, 0x4a, 0x49, 0x32, 0x8e, 0xe6, 0xbe, 0xbb, 0x42, 0x1d, 0xf8, 0x32, 0x70,
-	0x6e, 0x6c, 0x8f, 0xcc, 0xfd, 0x25, 0x49, 0x56, 0xb6, 0x43, 0x29, 0x09, 0x62, 0x9a, 0x72, 0xf5,
-	0x7b, 0xf8, 0x8b, 0xc0, 0xb9, 0xd1, 0xe5, 0x37, 0x4d, 0x7e, 0xba, 0x2c, 0x95, 0x0b, 0xaa, 0xd6,
-	0x9a, 0x40, 0x19, 0x93, 0x34, 0x5a, 0x24, 0x2e, 0x41, 0xbf, 0x04, 0x94, 0xc8, 0xb3, 0x1d, 0x3a,
-	0x01, 0x49, 0x63, 0xc7, 0x15, 0x2f, 0xa8, 0xe0, 0xa3, 0xec, 0xcb, 0x30, 0xfb, 0x80, 0x10, 0x94,
-	0x18, 0x4a, 0x1a, 0xca, 0xcf, 0xd2, 0xc4, 0x0b, 0xa8, 0x8d, 0x93, 0xc8, 0x5b, 0xb8, 0x24, 0xd1,
-	0x1d, 0xea, 0xa0, 0x0e, 0xec, 0xcd, 0x92, 0x68, 0x11, 0x73, 0x59, 0xd5, 0xce, 0xcf, 0x76, 0x79,
-	0x23, 0xb3, 0x02, 0x0b, 0xa8, 0x34, 0xef, 0xdf, 0x45, 0xa8, 0xf5, 0xa2, 0x30, 0x5d, 0x04, 0x9f,
-	0x2f, 0x0a, 0xbd, 0x81, 0x7a, 0xba, 0x98, 0xa6, 0x6e, 0xe2, 0xc7, 0x2c, 0x6a, 0x69, 0xb3, 0x70,
-	0x52, 0x3c, 0xad, 0x76, 0xbe, 0xd9, 0xc5, 0x3b, 0xd9, 0x00, 0x1a, 0x21, 0x4d, 0x56, 0xf8, 0x36,
-	0x2f, 0x32, 0xa0, 0xee, 0x0a, 0x83, 0xec, 0x20, 0xf2, 0xc8, 0xbc, 0x59, 0xe4, 0x11, 0x3e, 0xd9,
-	0x25, 0x4c, 0x5a, 0x3e, 0x60, 0x38, 0x5c, 0x73, 0x37, 0x6e, 0xe8, 0x02, 0x1a, 0x99, 0x98, 0x98,
-	0xc7, 0xb0, 0x59, 0xe2, 0x72, 0x9e, 0xdc, 0x23, 0x47, 0x04, 0x1b, 0x67, 0xfa, 0x65, 0xec, 0x31,
-	0x20, 0x8f, 0x38, 0x9e, 0x3d, 0xe7, 0x09, 0x91, 0x49, 0xdb, 0xe3, 0xee, 0x79, 0xb6, 0x4b, 0xda,
-	0x76, 0xf6, 0x60, 0xd5, 0xdb, 0xce, 0x27, 0x13, 0x32, 0x6b, 0x6d, 0x9e, 0xc5, 0xfb, 0xdc, 0xb6,
-	0x9f, 0xdf, 0xf7, 0x46, 0x92, 0xa6, 0xce, 0x8c, 0xf0, 0x6c, 0xae, 0x4a, 0x5e, 0x76, 0xb9, 0x2c,
-	0x95, 0x0f, 0x54, 0xad, 0xf5, 0x77, 0x05, 0x8e, 0xee, 0xb8, 0x96, 0x05, 0x93, 0x46, 0xb1, 0xef,
-	0x7e, 0x5a, 0x30, 0x39, 0x14, 0xe9, 0x77, 0x8a, 0xe4, 0x47, 0x9e, 0xb9, 0x5d, 0x92, 0x3b, 0x4a,
-	0xe9, 0xb7, 0x70, 0xa0, 0x79, 0x1e, 0xa3, 0xb1, 0x64, 0xbe, 0x8a, 0x52, 0x2a, 0xb3, 0x9d, 0x9f,
-	0x19, 0x2d, 0x8e, 0x12, 0xca, 0x95, 0xec, 0x61, 0x7e, 0x96, 0x8c, 0x7f, 0x53, 0xa0, 0x62, 0x84,
-	0x5e, 0x1c, 0xf9, 0x21, 0x4d, 0xd1, 0x77, 0xb0, 0x9f, 0xba, 0x57, 0x24, 0xc8, 0xaa, 0x7d, 0x67,
-	0x0c, 0xa5, 0xa2, 0x09, 0x07, 0x62, 0xc9, 0x80, 0xbe, 0x83, 0x8a, 0x23, 0x3e, 0x90, 0x2c, 0x2d,
-	0xbf, 0xba, 0x87, 0x1b, 0xaf, 0xd1, 0xd2, 0x92, 0x08, 0xf6, 0xbb, 0x49, 0x74, 0x4d, 0x92, 0xbc,
-	0x1c, 0x95, 0x75, 0x39, 0xa2, 0x06, 0x14, 0x7c, 0x4f, 0xda, 0x5f, 0xf0, 0x3d, 0xf4, 0x03, 0x54,
-	0x48, 0x66, 0x36, 0x4f, 0xdc, 0x6a, 0xe7, 0xd1, 0x2e, 0x75, 0xf9, 0xdb, 0xf0, 0x1a, 0x7f, 0x59,
-	0x2a, 0x97, 0x54, 0xad, 0xf5, 0x5f, 0x05, 0x2a, 0x63, 0x27, 0xa1, 0xbc, 0xab, 0x7e, 0x56, 0x04,
-	0xb7, 0x8d, 0xfa, 0x3d, 0x40, 0x4c, 0x92, 0xc0, 0x17, 0x11, 0x2d, 0xfe, 0x78, 0xc3, 0x1c, 0xe7,
-	0x28, 0xbc, 0xc1, 0x81, 0x3a, 0xb0, 0x3f, 0xe5, 0x2e, 0xe0, 0x25, 0x54, 0xed, 0x1c, 0xef, 0xe2,
-	0x15, 0x4e, 0xc2, 0x12, 0x79, 0x59, 0x2a, 0xef, 0xa9, 0x5a, 0xeb, 0xcf, 0xb0, 0xaf, 0xfb, 0x33,
-	0x92, 0xd2, 0x4f, 0x69, 0xd7, 0x02, 0xb9, 0xd1, 0xae, 0x8f, 0xa1, 0xec, 0x5e, 0x11, 0xf7, 0x3a,
-	0x5d, 0x04, 0xb2, 0x07, 0xe6, 0x77, 0x19, 0x9c, 0xff, 0x94, 0xe1, 0x70, 0xb2, 0x4a, 0x29, 0x09,
-	0x34, 0x4a, 0x13, 0x7f, 0xba, 0xa0, 0x04, 0xa9, 0x50, 0xa4, 0xce, 0x4c, 0x46, 0x89, 0x1d, 0x59,
-	0xe0, 0xae, 0xc9, 0x4a, 0x84, 0xbf, 0x82, 0xf9, 0x19, 0x3d, 0x02, 0x08, 0x44, 0x45, 0xd9, 0xbe,
-	0xc7, 0x7d, 0x52, 0xc1, 0x15, 0x49, 0x31, 0x59, 0x1c, 0xab, 0xd3, 0xc8, 0x5b, 0xd9, 0x1e, 0xb7,
-	0xe9, 0xbe, 0x77, 0x0b, 0xab, 0x31, 0x30, 0xb8, 0x7c, 0xab, 0x06, 0x75, 0xce, 0x4c, 0x42, 0x37,
-	0xf2, 0xfc, 0x70, 0xc6, 0x7b, 0x45, 0x63, 0x77, 0xec, 0x0c, 0x89, 0xc1, 0x35, 0xc6, 0x92, 0xdd,
-	0x50, 0x17, 0x6a, 0x99, 0x79, 0x1b, 0xfd, 0xe1, 0xeb, 0x5d, 0x12, 0x6e, 0x35, 0x86, 0x60, 0x7d,
-	0x41, 0x1a, 0x34, 0xa6, 0x51, 0x12, 0xda, 0xf9, 0xb4, 0x6d, 0x1e, 0xc8, 0x67, 0x88, 0x71, 0xdb,
-	0xce, 0xc6, 0x6d, 0xdb, 0xca, 0x10, 0xb8, 0xce, 0x38, 0xf2, 0x2b, 0xfa, 0x0a, 0x2a, 0x5c, 0x04,
-	0xaf, 0xdc, 0xb2, 0x08, 0x01, 0x23, 0x5c, 0xb0, 0xea, 0xed, 0xc1, 0x61, 0x4a, 0xa3, 0x84, 0x6c,
-	0x28, 0xa8, 0x7c, 0x54, 0x41, 0x83, 0xb3, 0xac, 0x35, 0x3c, 0x02, 0x10, 0x42, 0xb8, 0x0a, 0x10,
-	0x71, 0xe0, 0x14, 0xae, 0xe3, 0x0d, 0xeb, 0xbd, 0x72, 0xe6, 0xae, 0xd5, 0x54, 0x3f, 0xa6, 0xe6,
-	0xe2, 0x01, 0x3e, 0xca, 0xf8, 0xd6, 0xba, 0x9e, 0x40, 0xd5, 0x23, 0x73, 0x67, 0x65, 0xcf, 0xc9,
-	0x92, 0xcc, 0x9b, 0x35, 0x56, 0x20, 0x17, 0x0f, 0x30, 0x70, 0x62, 0x9f, 0xd1, 0xd0, 0x37, 0xd0,
-	0x48, 0x88, 0x4b, 0xfc, 0x98, 0xda, 0x57, 0x4e, 0xe8, 0xcd, 0x49, 0xb3, 0xce, 0x4d, 0xaa, 0x4b,
-	0xea, 0x05, 0x27, 0xa2, 0x27, 0x50, 0x8b, 0xb3, 0x12, 0x65, 0xf9, 0xd3, 0xe0, 0xb5, 0x56, 0xcd,
-	0x69, 0xa6, 0x87, 0x7e, 0x01, 0xea, 0x1a, 0x12, 0x7d, 0xf8, 0x90, 0x12, 0xda, 0x3c, 0x3c, 0x51,
-	0x4e, 0x8b, 0xf8, 0x30, 0xa7, 0x8f, 0x38, 0x19, 0xe9, 0xa0, 0xfa, 0xe1, 0xd2, 0x4f, 0xfd, 0xe9,
-	0x9c, 0xd8, 0x31, 0x49, 0xfc, 0xc8, 0x6b, 0xaa, 0xfc, 0x89, 0x3f, 0xbd, 0xf3, 0x44, 0x5d, 0x6e,
-	0x46, 0xf8, 0x30, 0x67, 0x19, 0x73, 0x0e, 0xa6, 0x70, 0x7b, 0x3d, 0x69, 0x1e, 0x71, 0xbb, 0x0e,
-	0xbd, 0xdb, 0xab, 0x09, 0xea, 0x41, 0x23, 0x96, 0xeb, 0x83, 0x2d, 0x86, 0x3d, 0xfa, 0x84, 0xee,
-	0x52, 0xcf, 0x78, 0xce, 0xf9, 0xd0, 0x7f, 0x0a, 0xf5, 0x2c, 0x45, 0x85, 0x8c, 0x2f, 0xb8, 0xa7,
-	0xb2, 0xbc, 0xcd, 0x41, 0x34, 0x71, 0x5c, 0x62, 0xbb, 0x51, 0x48, 0xc9, 0x0d, 0x6d, 0x3e, 0x14,
-	0x20, 0x4e, 0xec, 0x09, 0x1a, 0xfa, 0x0b, 0x3c, 0x8b, 0x92, 0xf8, 0xca, 0x09, 0x89, 0x67, 0xd3,
-	0xc4, 0x09, 0x53, 0xc7, 0xe5, 0x5e, 0x4b, 0x88, 0x1b, 0xf1, 0xe7, 0x48, 0x9f, 0x7c, 0xf9, 0x31,
-	0x9f, 0x3c, 0xc9, 0xc4, 0x58, 0x6b, 0x29, 0x58, 0x0a, 0x11, 0x5e, 0xea, 0xaa, 0xd0, 0x60, 0x79,
-	0xe4, 0xe5, 0xab, 0xdc, 0x65, 0xa9, 0xfc, 0x13, 0x55, 0x6b, 0xfd, 0xab, 0x00, 0x07, 0xb2, 0x92,
-	0x3e, 0xab, 0xe7, 0xbe, 0x83, 0xc6, 0x22, 0x25, 0x09, 0xf3, 0xbc, 0xe8, 0x43, 0x72, 0xd8, 0xb4,
-	0xef, 0x29, 0xd9, 0xf6, 0xbb, 0x94, 0x24, 0x79, 0xe3, 0x92, 0xcb, 0xd0, 0x62, 0x93, 0x86, 0x86,
-	0xa0, 0xa6, 0xbc, 0xbf, 0x6d, 0x08, 0x16, 0x63, 0xe5, 0xe9, 0xce, 0xe5, 0xea, 0x76, 0x2f, 0xc4,
-	0x87, 0xe9, 0x56, 0x73, 0x44, 0x50, 0x62, 0x7d, 0x86, 0x37, 0xb4, 0x1a, 0xe6, 0xe7, 0xe3, 0x57,
-	0x80, 0xee, 0x1a, 0xc2, 0xda, 0xe8, 0x35, 0x59, 0x65, 0x6d, 0xf4, 0x9a, 0xac, 0xd0, 0x43, 0xd8,
-	0x5b, 0x3a, 0xf3, 0x45, 0xb6, 0x8f, 0x8a, 0xcb, 0xf7, 0x85, 0xdf, 0x29, 0xb2, 0xd9, 0x8f, 0x00,
-	0xb4, 0x34, 0xf5, 0x67, 0x61, 0x40, 0x42, 0xca, 0x26, 0x61, 0x3e, 0xc5, 0xa4, 0x23, 0x77, 0x4e,
-	0xc2, 0x1c, 0x84, 0xd7, 0x78, 0xb1, 0x9b, 0x3e, 0xd7, 0x00, 0xd6, 0x13, 0x09, 0x95, 0xa1, 0x34,
-	0x1c, 0x0d, 0x0d, 0xf5, 0x01, 0x3b, 0x61, 0x43, 0xd3, 0x55, 0x05, 0x55, 0x60, 0xef, 0x8f, 0xd8,
-	0xb4, 0x0c, 0xb5, 0x80, 0x1a, 0x00, 0x8c, 0x68, 0x8b, 0x7b, 0xb1, 0xc5, 0x86, 0xe9, 0xab, 0xe7,
-	0xcf, 0x00, 0xd6, 0x7f, 0x01, 0xe8, 0x00, 0x8a, 0x96, 0x76, 0xae, 0x3e, 0x60, 0x87, 0xc9, 0xdb,
-	0xbe, 0xaa, 0xb4, 0x98, 0xa2, 0x57, 0xcf, 0xbf, 0xcd, 0x77, 0x60, 0xb1, 0x3b, 0x36, 0x00, 0x7a,
-	0xfd, 0x77, 0x13, 0xcb, 0xc0, 0xe6, 0x90, 0xc1, 0x55, 0xa8, 0x75, 0xf1, 0x48, 0xd3, 0x7b, 0xda,
-	0xc4, 0x62, 0x94, 0x8c, 0x0f, 0x43, 0xfd, 0xd6, 0xe6, 0x88, 0x00, 0xf6, 0xb1, 0x31, 0x79, 0x37,
-	0x60, 0x56, 0xd6, 0xa0, 0x3c, 0xee, 0x6b, 0xef, 0xbb, 0x5a, 0xef, 0x8d, 0xaa, 0xa0, 0x2a, 0x1c,
-	0xe8, 0xe6, 0xa4, 0xa7, 0x61, 0x5d, 0x2d, 0xa0, 0x87, 0xa0, 0x5a, 0x1a, 0x3e, 0x37, 0x2c, 0xdb,
-	0x32, 0x07, 0xc6, 0xc4, 0xd2, 0x06, 0xe3, 0xdc, 0xe2, 0x97, 0x80, 0xee, 0x6e, 0x7c, 0x4c, 0xb0,
-	0xd6, 0xb3, 0xcc, 0x3f, 0x30, 0xc1, 0x55, 0x38, 0x18, 0x6b, 0x93, 0x09, 0xbb, 0x64, 0x86, 0x7c,
-	0x0f, 0xf5, 0x5b, 0xeb, 0x0f, 0x73, 0x91, 0x39, 0x5e, 0xfe, 0x5a, 0x38, 0xcb, 0x1c, 0x2f, 0xbf,
-	0x55, 0x15, 0x74, 0x08, 0x55, 0x7d, 0x34, 0xd0, 0xcc, 0xa1, 0x3d, 0xd4, 0x06, 0x86, 0x5a, 0x68,
-	0xb1, 0x19, 0xfa, 0xea, 0xb9, 0x01, 0xd5, 0x2d, 0x4d, 0xc3, 0x11, 0x1e, 0x68, 0x7d, 0xc1, 0xfb,
-	0xda, 0x7c, 0x3d, 0x12, 0x8e, 0xd6, 0x8d, 0xbe, 0xf6, 0x5e, 0x2d, 0x30, 0x31, 0x16, 0xd6, 0x86,
-	0x13, 0x66, 0xcf, 0x68, 0x98, 0xdb, 0xfd, 0x2b, 0x80, 0xf5, 0x00, 0x67, 0xf8, 0x1e, 0xee, 0xbd,
-	0xec, 0x08, 0x5f, 0x0f, 0xf4, 0xdf, 0xa8, 0x0a, 0x93, 0x36, 0xb9, 0xd0, 0x5e, 0xe4, 0x8a, 0x4f,
-	0xa1, 0x9c, 0xcf, 0xbb, 0x1a, 0x94, 0x4d, 0xdd, 0x18, 0x5a, 0xa6, 0xf5, 0x5e, 0xe8, 0x3d, 0xff,
-	0x93, 0x39, 0xce, 0x9f, 0xa7, 0xc1, 0xd1, 0xdb, 0x05, 0x49, 0x56, 0xa2, 0x5f, 0x4a, 0x5f, 0xd7,
-	0xa1, 0xd2, 0x35, 0xce, 0xcd, 0xe1, 0x50, 0xc4, 0xe8, 0x00, 0x8a, 0xc6, 0x90, 0xe5, 0x44, 0x03,
-	0x80, 0x79, 0xd5, 0x1e, 0x8f, 0xcc, 0xa1, 0x95, 0x29, 0xeb, 0xfe, 0x00, 0x3b, 0xfe, 0x4c, 0xbb,
-	0xe5, 0xc1, 0x5b, 0x3d, 0x0a, 0x1c, 0x3f, 0x1c, 0x2b, 0xff, 0x50, 0x94, 0xff, 0x29, 0xca, 0x3f,
-	0x0b, 0x3b, 0x60, 0xd3, 0x7d, 0xde, 0x6a, 0x5e, 0xfe, 0x3f, 0x00, 0x00, 0xff, 0xff, 0x4c, 0x48,
-	0x76, 0xa2, 0x02, 0x0f, 0x00, 0x00,
-}
diff --git a/golang/protocol/v1/service.pb.go b/golang/protocol/v1/service.pb.go
deleted file mode 100644
index 19f8d55..0000000
--- a/golang/protocol/v1/service.pb.go
+++ /dev/null
@@ -1,3342 +0,0 @@
-// Code generated by protoc-gen-go. DO NOT EDIT.
-// source: apache/rocketmq/v1/service.proto
-
-package v1
-
-import (
-	context "context"
-	fmt "fmt"
-	math "math"
-
-	proto "github.com/golang/protobuf/proto"
-	errdetails "google.golang.org/genproto/googleapis/rpc/errdetails"
-	status "google.golang.org/genproto/googleapis/rpc/status"
-	grpc "google.golang.org/grpc"
-	codes "google.golang.org/grpc/codes"
-	status1 "google.golang.org/grpc/status"
-	durationpb "google.golang.org/protobuf/types/known/durationpb"
-	timestamppb "google.golang.org/protobuf/types/known/timestamppb"
-)
-
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
-
-type EndTransactionRequest_TransactionResolution int32
-
-const (
-	EndTransactionRequest_COMMIT   EndTransactionRequest_TransactionResolution = 0
-	EndTransactionRequest_ROLLBACK EndTransactionRequest_TransactionResolution = 1
-)
-
-var EndTransactionRequest_TransactionResolution_name = map[int32]string{
-	0: "COMMIT",
-	1: "ROLLBACK",
-}
-
-var EndTransactionRequest_TransactionResolution_value = map[string]int32{
-	"COMMIT":   0,
-	"ROLLBACK": 1,
-}
-
-func (x EndTransactionRequest_TransactionResolution) String() string {
-	return proto.EnumName(EndTransactionRequest_TransactionResolution_name, int32(x))
-}
-
-func (EndTransactionRequest_TransactionResolution) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{19, 0}
-}
-
-type EndTransactionRequest_Source int32
-
-const (
-	EndTransactionRequest_CLIENT       EndTransactionRequest_Source = 0
-	EndTransactionRequest_SERVER_CHECK EndTransactionRequest_Source = 1
-)
-
-var EndTransactionRequest_Source_name = map[int32]string{
-	0: "CLIENT",
-	1: "SERVER_CHECK",
-}
-
-var EndTransactionRequest_Source_value = map[string]int32{
-	"CLIENT":       0,
-	"SERVER_CHECK": 1,
-}
-
-func (x EndTransactionRequest_Source) String() string {
-	return proto.EnumName(EndTransactionRequest_Source_name, int32(x))
-}
-
-func (EndTransactionRequest_Source) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{19, 1}
-}
-
-type ResponseCommon struct {
-	Status               *status.Status          `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
-	RequestInfo          *errdetails.RequestInfo `protobuf:"bytes,2,opt,name=request_info,json=requestInfo,proto3" json:"request_info,omitempty"`
-	Help                 *errdetails.Help        `protobuf:"bytes,3,opt,name=help,proto3" json:"help,omitempty"`
-	RetryInfo            *errdetails.RetryInfo   `protobuf:"bytes,4,opt,name=retry_info,json=retryInfo,proto3" json:"retry_info,omitempty"`
-	DebugInfo            *errdetails.DebugInfo   `protobuf:"bytes,5,opt,name=debug_info,json=debugInfo,proto3" json:"debug_info,omitempty"`
-	ErrorInfo            *errdetails.ErrorInfo   `protobuf:"bytes,6,opt,name=error_info,json=errorInfo,proto3" json:"error_info,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
-}
-
-func (m *ResponseCommon) Reset()         { *m = ResponseCommon{} }
-func (m *ResponseCommon) String() string { return proto.CompactTextString(m) }
-func (*ResponseCommon) ProtoMessage()    {}
-func (*ResponseCommon) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{0}
-}
-
-func (m *ResponseCommon) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ResponseCommon.Unmarshal(m, b)
-}
-func (m *ResponseCommon) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ResponseCommon.Marshal(b, m, deterministic)
-}
-func (m *ResponseCommon) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ResponseCommon.Merge(m, src)
-}
-func (m *ResponseCommon) XXX_Size() int {
-	return xxx_messageInfo_ResponseCommon.Size(m)
-}
-func (m *ResponseCommon) XXX_DiscardUnknown() {
-	xxx_messageInfo_ResponseCommon.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ResponseCommon proto.InternalMessageInfo
-
-func (m *ResponseCommon) GetStatus() *status.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-func (m *ResponseCommon) GetRequestInfo() *errdetails.RequestInfo {
-	if m != nil {
-		return m.RequestInfo
-	}
-	return nil
-}
-
-func (m *ResponseCommon) GetHelp() *errdetails.Help {
-	if m != nil {
-		return m.Help
-	}
-	return nil
-}
-
-func (m *ResponseCommon) GetRetryInfo() *errdetails.RetryInfo {
-	if m != nil {
-		return m.RetryInfo
-	}
-	return nil
-}
-
-func (m *ResponseCommon) GetDebugInfo() *errdetails.DebugInfo {
-	if m != nil {
-		return m.DebugInfo
-	}
-	return nil
-}
-
-func (m *ResponseCommon) GetErrorInfo() *errdetails.ErrorInfo {
-	if m != nil {
-		return m.ErrorInfo
-	}
-	return nil
-}
-
-// Topics are destination of messages to publish to or subscribe from. Similar
-// to domain names, they will be addressable after resolution through the
-// provided access point.
-//
-// Access points are usually the addresses of name servers, which fulfill
-// service discovery, load-balancing and other auxillary services. Name servers
-// receive periodic heartbeats from affiliate brokers and erase those which
-// failed to maintain alive status.
-//
-// Name servers answer queries of QueryRouteRequest, responding clients with
-// addressable partitions, which they may directly publish messages to or
-// subscribe messages from.
-//
-// QueryRouteRequest shall include source endpoints, aka, configured
-// access-point, which annotates tenant-id, instance-id or other
-// vendor-specific settings. Purpose-built name servers may respond customized
-// results based on these particular requirements.
-type QueryRouteRequest struct {
-	Topic                *Resource  `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
-	Endpoints            *Endpoints `protobuf:"bytes,2,opt,name=endpoints,proto3" json:"endpoints,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *QueryRouteRequest) Reset()         { *m = QueryRouteRequest{} }
-func (m *QueryRouteRequest) String() string { return proto.CompactTextString(m) }
-func (*QueryRouteRequest) ProtoMessage()    {}
-func (*QueryRouteRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{1}
-}
-
-func (m *QueryRouteRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryRouteRequest.Unmarshal(m, b)
-}
-func (m *QueryRouteRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryRouteRequest.Marshal(b, m, deterministic)
-}
-func (m *QueryRouteRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryRouteRequest.Merge(m, src)
-}
-func (m *QueryRouteRequest) XXX_Size() int {
-	return xxx_messageInfo_QueryRouteRequest.Size(m)
-}
-func (m *QueryRouteRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryRouteRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryRouteRequest proto.InternalMessageInfo
-
-func (m *QueryRouteRequest) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *QueryRouteRequest) GetEndpoints() *Endpoints {
-	if m != nil {
-		return m.Endpoints
-	}
-	return nil
-}
-
-type QueryRouteResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	Partitions           []*Partition    `protobuf:"bytes,2,rep,name=partitions,proto3" json:"partitions,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *QueryRouteResponse) Reset()         { *m = QueryRouteResponse{} }
-func (m *QueryRouteResponse) String() string { return proto.CompactTextString(m) }
-func (*QueryRouteResponse) ProtoMessage()    {}
-func (*QueryRouteResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{2}
-}
-
-func (m *QueryRouteResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryRouteResponse.Unmarshal(m, b)
-}
-func (m *QueryRouteResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryRouteResponse.Marshal(b, m, deterministic)
-}
-func (m *QueryRouteResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryRouteResponse.Merge(m, src)
-}
-func (m *QueryRouteResponse) XXX_Size() int {
-	return xxx_messageInfo_QueryRouteResponse.Size(m)
-}
-func (m *QueryRouteResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryRouteResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryRouteResponse proto.InternalMessageInfo
-
-func (m *QueryRouteResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func (m *QueryRouteResponse) GetPartitions() []*Partition {
-	if m != nil {
-		return m.Partitions
-	}
-	return nil
-}
-
-type SendMessageRequest struct {
-	Message              *Message   `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"`
-	Partition            *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *SendMessageRequest) Reset()         { *m = SendMessageRequest{} }
-func (m *SendMessageRequest) String() string { return proto.CompactTextString(m) }
-func (*SendMessageRequest) ProtoMessage()    {}
-func (*SendMessageRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{3}
-}
-
-func (m *SendMessageRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SendMessageRequest.Unmarshal(m, b)
-}
-func (m *SendMessageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SendMessageRequest.Marshal(b, m, deterministic)
-}
-func (m *SendMessageRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SendMessageRequest.Merge(m, src)
-}
-func (m *SendMessageRequest) XXX_Size() int {
-	return xxx_messageInfo_SendMessageRequest.Size(m)
-}
-func (m *SendMessageRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_SendMessageRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SendMessageRequest proto.InternalMessageInfo
-
-func (m *SendMessageRequest) GetMessage() *Message {
-	if m != nil {
-		return m.Message
-	}
-	return nil
-}
-
-func (m *SendMessageRequest) GetPartition() *Partition {
-	if m != nil {
-		return m.Partition
-	}
-	return nil
-}
-
-type SendMessageResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	MessageId            string          `protobuf:"bytes,2,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	TransactionId        string          `protobuf:"bytes,3,opt,name=transaction_id,json=transactionId,proto3" json:"transaction_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *SendMessageResponse) Reset()         { *m = SendMessageResponse{} }
-func (m *SendMessageResponse) String() string { return proto.CompactTextString(m) }
-func (*SendMessageResponse) ProtoMessage()    {}
-func (*SendMessageResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{4}
-}
-
-func (m *SendMessageResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SendMessageResponse.Unmarshal(m, b)
-}
-func (m *SendMessageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SendMessageResponse.Marshal(b, m, deterministic)
-}
-func (m *SendMessageResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SendMessageResponse.Merge(m, src)
-}
-func (m *SendMessageResponse) XXX_Size() int {
-	return xxx_messageInfo_SendMessageResponse.Size(m)
-}
-func (m *SendMessageResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_SendMessageResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SendMessageResponse proto.InternalMessageInfo
-
-func (m *SendMessageResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func (m *SendMessageResponse) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
-	}
-	return ""
-}
-
-func (m *SendMessageResponse) GetTransactionId() string {
-	if m != nil {
-		return m.TransactionId
-	}
-	return ""
-}
-
-type QueryAssignmentRequest struct {
-	Topic    *Resource `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
-	Group    *Resource `protobuf:"bytes,2,opt,name=group,proto3" json:"group,omitempty"`
-	ClientId string    `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	// Service access point
-	Endpoints            *Endpoints `protobuf:"bytes,4,opt,name=endpoints,proto3" json:"endpoints,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *QueryAssignmentRequest) Reset()         { *m = QueryAssignmentRequest{} }
-func (m *QueryAssignmentRequest) String() string { return proto.CompactTextString(m) }
-func (*QueryAssignmentRequest) ProtoMessage()    {}
-func (*QueryAssignmentRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{5}
-}
-
-func (m *QueryAssignmentRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryAssignmentRequest.Unmarshal(m, b)
-}
-func (m *QueryAssignmentRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryAssignmentRequest.Marshal(b, m, deterministic)
-}
-func (m *QueryAssignmentRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryAssignmentRequest.Merge(m, src)
-}
-func (m *QueryAssignmentRequest) XXX_Size() int {
-	return xxx_messageInfo_QueryAssignmentRequest.Size(m)
-}
-func (m *QueryAssignmentRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryAssignmentRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryAssignmentRequest proto.InternalMessageInfo
-
-func (m *QueryAssignmentRequest) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *QueryAssignmentRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *QueryAssignmentRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-func (m *QueryAssignmentRequest) GetEndpoints() *Endpoints {
-	if m != nil {
-		return m.Endpoints
-	}
-	return nil
-}
-
-type QueryAssignmentResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	Assignments          []*Assignment   `protobuf:"bytes,2,rep,name=assignments,proto3" json:"assignments,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *QueryAssignmentResponse) Reset()         { *m = QueryAssignmentResponse{} }
-func (m *QueryAssignmentResponse) String() string { return proto.CompactTextString(m) }
-func (*QueryAssignmentResponse) ProtoMessage()    {}
-func (*QueryAssignmentResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{6}
-}
-
-func (m *QueryAssignmentResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryAssignmentResponse.Unmarshal(m, b)
-}
-func (m *QueryAssignmentResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryAssignmentResponse.Marshal(b, m, deterministic)
-}
-func (m *QueryAssignmentResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryAssignmentResponse.Merge(m, src)
-}
-func (m *QueryAssignmentResponse) XXX_Size() int {
-	return xxx_messageInfo_QueryAssignmentResponse.Size(m)
-}
-func (m *QueryAssignmentResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryAssignmentResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryAssignmentResponse proto.InternalMessageInfo
-
-func (m *QueryAssignmentResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func (m *QueryAssignmentResponse) GetAssignments() []*Assignment {
-	if m != nil {
-		return m.Assignments
-	}
-	return nil
-}
-
-type ReceiveMessageRequest struct {
-	Group                   *Resource              `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	ClientId                string                 `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	Partition               *Partition             `protobuf:"bytes,3,opt,name=partition,proto3" json:"partition,omitempty"`
-	FilterExpression        *FilterExpression      `protobuf:"bytes,4,opt,name=filter_expression,json=filterExpression,proto3" json:"filter_expression,omitempty"`
-	ConsumePolicy           ConsumePolicy          `protobuf:"varint,5,opt,name=consume_policy,json=consumePolicy,proto3,enum=apache.rocketmq.v1.ConsumePolicy" json:"consume_policy,omitempty"`
-	InitializationTimestamp *timestamppb.Timestamp `protobuf:"bytes,6,opt,name=initialization_timestamp,json=initializationTimestamp,proto3" json:"initialization_timestamp,omitempty"`
-	BatchSize               int32                  `protobuf:"varint,7,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"`
-	InvisibleDuration       *durationpb.Duration   `protobuf:"bytes,8,opt,name=invisible_duration,json=invisibleDuration,proto3" json:"invisible_duration,omitempty"`
-	AwaitTime               *durationpb.Duration   `protobuf:"bytes,9,opt,name=await_time,json=awaitTime,proto3" json:"await_time,omitempty"`
-	FifoFlag                bool                   `protobuf:"varint,10,opt,name=fifo_flag,json=fifoFlag,proto3" json:"fifo_flag,omitempty"`
-	XXX_NoUnkeyedLiteral    struct{}               `json:"-"`
-	XXX_unrecognized        []byte                 `json:"-"`
-	XXX_sizecache           int32                  `json:"-"`
-}
-
-func (m *ReceiveMessageRequest) Reset()         { *m = ReceiveMessageRequest{} }
-func (m *ReceiveMessageRequest) String() string { return proto.CompactTextString(m) }
-func (*ReceiveMessageRequest) ProtoMessage()    {}
-func (*ReceiveMessageRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{7}
-}
-
-func (m *ReceiveMessageRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReceiveMessageRequest.Unmarshal(m, b)
-}
-func (m *ReceiveMessageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReceiveMessageRequest.Marshal(b, m, deterministic)
-}
-func (m *ReceiveMessageRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReceiveMessageRequest.Merge(m, src)
-}
-func (m *ReceiveMessageRequest) XXX_Size() int {
-	return xxx_messageInfo_ReceiveMessageRequest.Size(m)
-}
-func (m *ReceiveMessageRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReceiveMessageRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReceiveMessageRequest proto.InternalMessageInfo
-
-func (m *ReceiveMessageRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *ReceiveMessageRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-func (m *ReceiveMessageRequest) GetPartition() *Partition {
-	if m != nil {
-		return m.Partition
-	}
-	return nil
-}
-
-func (m *ReceiveMessageRequest) GetFilterExpression() *FilterExpression {
-	if m != nil {
-		return m.FilterExpression
-	}
-	return nil
-}
-
-func (m *ReceiveMessageRequest) GetConsumePolicy() ConsumePolicy {
-	if m != nil {
-		return m.ConsumePolicy
-	}
-	return ConsumePolicy_RESUME
-}
-
-func (m *ReceiveMessageRequest) GetInitializationTimestamp() *timestamppb.Timestamp {
-	if m != nil {
-		return m.InitializationTimestamp
-	}
-	return nil
-}
-
-func (m *ReceiveMessageRequest) GetBatchSize() int32 {
-	if m != nil {
-		return m.BatchSize
-	}
-	return 0
-}
-
-func (m *ReceiveMessageRequest) GetInvisibleDuration() *durationpb.Duration {
-	if m != nil {
-		return m.InvisibleDuration
-	}
-	return nil
-}
-
-func (m *ReceiveMessageRequest) GetAwaitTime() *durationpb.Duration {
-	if m != nil {
-		return m.AwaitTime
-	}
-	return nil
-}
-
-func (m *ReceiveMessageRequest) GetFifoFlag() bool {
-	if m != nil {
-		return m.FifoFlag
-	}
-	return false
-}
-
-type ReceiveMessageResponse struct {
-	Common               *ResponseCommon        `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	Messages             []*Message             `protobuf:"bytes,2,rep,name=messages,proto3" json:"messages,omitempty"`
-	DeliveryTimestamp    *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=delivery_timestamp,json=deliveryTimestamp,proto3" json:"delivery_timestamp,omitempty"`
-	InvisibleDuration    *durationpb.Duration   `protobuf:"bytes,4,opt,name=invisible_duration,json=invisibleDuration,proto3" json:"invisible_duration,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *ReceiveMessageResponse) Reset()         { *m = ReceiveMessageResponse{} }
-func (m *ReceiveMessageResponse) String() string { return proto.CompactTextString(m) }
-func (*ReceiveMessageResponse) ProtoMessage()    {}
-func (*ReceiveMessageResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{8}
-}
-
-func (m *ReceiveMessageResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReceiveMessageResponse.Unmarshal(m, b)
-}
-func (m *ReceiveMessageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReceiveMessageResponse.Marshal(b, m, deterministic)
-}
-func (m *ReceiveMessageResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReceiveMessageResponse.Merge(m, src)
-}
-func (m *ReceiveMessageResponse) XXX_Size() int {
-	return xxx_messageInfo_ReceiveMessageResponse.Size(m)
-}
-func (m *ReceiveMessageResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReceiveMessageResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReceiveMessageResponse proto.InternalMessageInfo
-
-func (m *ReceiveMessageResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func (m *ReceiveMessageResponse) GetMessages() []*Message {
-	if m != nil {
-		return m.Messages
-	}
-	return nil
-}
-
-func (m *ReceiveMessageResponse) GetDeliveryTimestamp() *timestamppb.Timestamp {
-	if m != nil {
-		return m.DeliveryTimestamp
-	}
-	return nil
-}
-
-func (m *ReceiveMessageResponse) GetInvisibleDuration() *durationpb.Duration {
-	if m != nil {
-		return m.InvisibleDuration
-	}
-	return nil
-}
-
-type AckMessageRequest struct {
-	Group    *Resource `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	Topic    *Resource `protobuf:"bytes,2,opt,name=topic,proto3" json:"topic,omitempty"`
-	ClientId string    `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	// Types that are valid to be assigned to Handle:
-	//	*AckMessageRequest_ReceiptHandle
-	//	*AckMessageRequest_Offset
-	Handle               isAckMessageRequest_Handle `protobuf_oneof:"handle"`
-	MessageId            string                     `protobuf:"bytes,6,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
-}
-
-func (m *AckMessageRequest) Reset()         { *m = AckMessageRequest{} }
-func (m *AckMessageRequest) String() string { return proto.CompactTextString(m) }
-func (*AckMessageRequest) ProtoMessage()    {}
-func (*AckMessageRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{9}
-}
-
-func (m *AckMessageRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AckMessageRequest.Unmarshal(m, b)
-}
-func (m *AckMessageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AckMessageRequest.Marshal(b, m, deterministic)
-}
-func (m *AckMessageRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AckMessageRequest.Merge(m, src)
-}
-func (m *AckMessageRequest) XXX_Size() int {
-	return xxx_messageInfo_AckMessageRequest.Size(m)
-}
-func (m *AckMessageRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_AckMessageRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AckMessageRequest proto.InternalMessageInfo
-
-func (m *AckMessageRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *AckMessageRequest) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *AckMessageRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-type isAckMessageRequest_Handle interface {
-	isAckMessageRequest_Handle()
-}
-
-type AckMessageRequest_ReceiptHandle struct {
-	ReceiptHandle string `protobuf:"bytes,4,opt,name=receipt_handle,json=receiptHandle,proto3,oneof"`
-}
-
-type AckMessageRequest_Offset struct {
-	Offset int64 `protobuf:"varint,5,opt,name=offset,proto3,oneof"`
-}
-
-func (*AckMessageRequest_ReceiptHandle) isAckMessageRequest_Handle() {}
-
-func (*AckMessageRequest_Offset) isAckMessageRequest_Handle() {}
-
-func (m *AckMessageRequest) GetHandle() isAckMessageRequest_Handle {
-	if m != nil {
-		return m.Handle
-	}
-	return nil
-}
-
-func (m *AckMessageRequest) GetReceiptHandle() string {
-	if x, ok := m.GetHandle().(*AckMessageRequest_ReceiptHandle); ok {
-		return x.ReceiptHandle
-	}
-	return ""
-}
-
-func (m *AckMessageRequest) GetOffset() int64 {
-	if x, ok := m.GetHandle().(*AckMessageRequest_Offset); ok {
-		return x.Offset
-	}
-	return 0
-}
-
-func (m *AckMessageRequest) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
-	}
-	return ""
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*AckMessageRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*AckMessageRequest_ReceiptHandle)(nil),
-		(*AckMessageRequest_Offset)(nil),
-	}
-}
-
-type AckMessageResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *AckMessageResponse) Reset()         { *m = AckMessageResponse{} }
-func (m *AckMessageResponse) String() string { return proto.CompactTextString(m) }
-func (*AckMessageResponse) ProtoMessage()    {}
-func (*AckMessageResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{10}
-}
-
-func (m *AckMessageResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AckMessageResponse.Unmarshal(m, b)
-}
-func (m *AckMessageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AckMessageResponse.Marshal(b, m, deterministic)
-}
-func (m *AckMessageResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AckMessageResponse.Merge(m, src)
-}
-func (m *AckMessageResponse) XXX_Size() int {
-	return xxx_messageInfo_AckMessageResponse.Size(m)
-}
-func (m *AckMessageResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_AckMessageResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AckMessageResponse proto.InternalMessageInfo
-
-func (m *AckMessageResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type NackMessageRequest struct {
-	Group                *Resource `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	Topic                *Resource `protobuf:"bytes,2,opt,name=topic,proto3" json:"topic,omitempty"`
-	ClientId             string    `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	ReceiptHandle        string    `protobuf:"bytes,4,opt,name=receipt_handle,json=receiptHandle,proto3" json:"receipt_handle,omitempty"`
-	MessageId            string    `protobuf:"bytes,5,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	DeliveryAttempt      int32     `protobuf:"varint,6,opt,name=delivery_attempt,json=deliveryAttempt,proto3" json:"delivery_attempt,omitempty"`
-	MaxDeliveryAttempts  int32     `protobuf:"varint,7,opt,name=max_delivery_attempts,json=maxDeliveryAttempts,proto3" json:"max_delivery_attempts,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *NackMessageRequest) Reset()         { *m = NackMessageRequest{} }
-func (m *NackMessageRequest) String() string { return proto.CompactTextString(m) }
-func (*NackMessageRequest) ProtoMessage()    {}
-func (*NackMessageRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{11}
-}
-
-func (m *NackMessageRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_NackMessageRequest.Unmarshal(m, b)
-}
-func (m *NackMessageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_NackMessageRequest.Marshal(b, m, deterministic)
-}
-func (m *NackMessageRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_NackMessageRequest.Merge(m, src)
-}
-func (m *NackMessageRequest) XXX_Size() int {
-	return xxx_messageInfo_NackMessageRequest.Size(m)
-}
-func (m *NackMessageRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_NackMessageRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_NackMessageRequest proto.InternalMessageInfo
-
-func (m *NackMessageRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *NackMessageRequest) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *NackMessageRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-func (m *NackMessageRequest) GetReceiptHandle() string {
-	if m != nil {
-		return m.ReceiptHandle
-	}
-	return ""
-}
-
-func (m *NackMessageRequest) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
-	}
-	return ""
-}
-
-func (m *NackMessageRequest) GetDeliveryAttempt() int32 {
-	if m != nil {
-		return m.DeliveryAttempt
-	}
-	return 0
-}
-
-func (m *NackMessageRequest) GetMaxDeliveryAttempts() int32 {
-	if m != nil {
-		return m.MaxDeliveryAttempts
-	}
-	return 0
-}
-
-type NackMessageResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *NackMessageResponse) Reset()         { *m = NackMessageResponse{} }
-func (m *NackMessageResponse) String() string { return proto.CompactTextString(m) }
-func (*NackMessageResponse) ProtoMessage()    {}
-func (*NackMessageResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{12}
-}
-
-func (m *NackMessageResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_NackMessageResponse.Unmarshal(m, b)
-}
-func (m *NackMessageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_NackMessageResponse.Marshal(b, m, deterministic)
-}
-func (m *NackMessageResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_NackMessageResponse.Merge(m, src)
-}
-func (m *NackMessageResponse) XXX_Size() int {
-	return xxx_messageInfo_NackMessageResponse.Size(m)
-}
-func (m *NackMessageResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_NackMessageResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_NackMessageResponse proto.InternalMessageInfo
-
-func (m *NackMessageResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type ForwardMessageToDeadLetterQueueRequest struct {
-	Group                *Resource `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	Topic                *Resource `protobuf:"bytes,2,opt,name=topic,proto3" json:"topic,omitempty"`
-	ClientId             string    `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	ReceiptHandle        string    `protobuf:"bytes,4,opt,name=receipt_handle,json=receiptHandle,proto3" json:"receipt_handle,omitempty"`
-	MessageId            string    `protobuf:"bytes,5,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	DeliveryAttempt      int32     `protobuf:"varint,6,opt,name=delivery_attempt,json=deliveryAttempt,proto3" json:"delivery_attempt,omitempty"`
-	MaxDeliveryAttempts  int32     `protobuf:"varint,7,opt,name=max_delivery_attempts,json=maxDeliveryAttempts,proto3" json:"max_delivery_attempts,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) Reset() {
-	*m = ForwardMessageToDeadLetterQueueRequest{}
-}
-func (m *ForwardMessageToDeadLetterQueueRequest) String() string { return proto.CompactTextString(m) }
-func (*ForwardMessageToDeadLetterQueueRequest) ProtoMessage()    {}
-func (*ForwardMessageToDeadLetterQueueRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{13}
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ForwardMessageToDeadLetterQueueRequest.Unmarshal(m, b)
-}
-func (m *ForwardMessageToDeadLetterQueueRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ForwardMessageToDeadLetterQueueRequest.Marshal(b, m, deterministic)
-}
-func (m *ForwardMessageToDeadLetterQueueRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ForwardMessageToDeadLetterQueueRequest.Merge(m, src)
-}
-func (m *ForwardMessageToDeadLetterQueueRequest) XXX_Size() int {
-	return xxx_messageInfo_ForwardMessageToDeadLetterQueueRequest.Size(m)
-}
-func (m *ForwardMessageToDeadLetterQueueRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ForwardMessageToDeadLetterQueueRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ForwardMessageToDeadLetterQueueRequest proto.InternalMessageInfo
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetTopic() *Resource {
-	if m != nil {
-		return m.Topic
-	}
-	return nil
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetReceiptHandle() string {
-	if m != nil {
-		return m.ReceiptHandle
-	}
-	return ""
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
-	}
-	return ""
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetDeliveryAttempt() int32 {
-	if m != nil {
-		return m.DeliveryAttempt
-	}
-	return 0
-}
-
-func (m *ForwardMessageToDeadLetterQueueRequest) GetMaxDeliveryAttempts() int32 {
-	if m != nil {
-		return m.MaxDeliveryAttempts
-	}
-	return 0
-}
-
-type ForwardMessageToDeadLetterQueueResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *ForwardMessageToDeadLetterQueueResponse) Reset() {
-	*m = ForwardMessageToDeadLetterQueueResponse{}
-}
-func (m *ForwardMessageToDeadLetterQueueResponse) String() string { return proto.CompactTextString(m) }
-func (*ForwardMessageToDeadLetterQueueResponse) ProtoMessage()    {}
-func (*ForwardMessageToDeadLetterQueueResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{14}
-}
-
-func (m *ForwardMessageToDeadLetterQueueResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ForwardMessageToDeadLetterQueueResponse.Unmarshal(m, b)
-}
-func (m *ForwardMessageToDeadLetterQueueResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ForwardMessageToDeadLetterQueueResponse.Marshal(b, m, deterministic)
-}
-func (m *ForwardMessageToDeadLetterQueueResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ForwardMessageToDeadLetterQueueResponse.Merge(m, src)
-}
-func (m *ForwardMessageToDeadLetterQueueResponse) XXX_Size() int {
-	return xxx_messageInfo_ForwardMessageToDeadLetterQueueResponse.Size(m)
-}
-func (m *ForwardMessageToDeadLetterQueueResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ForwardMessageToDeadLetterQueueResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ForwardMessageToDeadLetterQueueResponse proto.InternalMessageInfo
-
-func (m *ForwardMessageToDeadLetterQueueResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type HeartbeatRequest struct {
-	ClientId string `protobuf:"bytes,1,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	// Types that are valid to be assigned to ClientData:
-	//	*HeartbeatRequest_ProducerData
-	//	*HeartbeatRequest_ConsumerData
-	ClientData           isHeartbeatRequest_ClientData `protobuf_oneof:"client_data"`
-	FifoFlag             bool                          `protobuf:"varint,4,opt,name=fifo_flag,json=fifoFlag,proto3" json:"fifo_flag,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
-	XXX_unrecognized     []byte                        `json:"-"`
-	XXX_sizecache        int32                         `json:"-"`
-}
-
-func (m *HeartbeatRequest) Reset()         { *m = HeartbeatRequest{} }
-func (m *HeartbeatRequest) String() string { return proto.CompactTextString(m) }
-func (*HeartbeatRequest) ProtoMessage()    {}
-func (*HeartbeatRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{15}
-}
-
-func (m *HeartbeatRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_HeartbeatRequest.Unmarshal(m, b)
-}
-func (m *HeartbeatRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_HeartbeatRequest.Marshal(b, m, deterministic)
-}
-func (m *HeartbeatRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_HeartbeatRequest.Merge(m, src)
-}
-func (m *HeartbeatRequest) XXX_Size() int {
-	return xxx_messageInfo_HeartbeatRequest.Size(m)
-}
-func (m *HeartbeatRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_HeartbeatRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_HeartbeatRequest proto.InternalMessageInfo
-
-func (m *HeartbeatRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-type isHeartbeatRequest_ClientData interface {
-	isHeartbeatRequest_ClientData()
-}
-
-type HeartbeatRequest_ProducerData struct {
-	ProducerData *ProducerData `protobuf:"bytes,2,opt,name=producer_data,json=producerData,proto3,oneof"`
-}
-
-type HeartbeatRequest_ConsumerData struct {
-	ConsumerData *ConsumerData `protobuf:"bytes,3,opt,name=consumer_data,json=consumerData,proto3,oneof"`
-}
-
-func (*HeartbeatRequest_ProducerData) isHeartbeatRequest_ClientData() {}
-
-func (*HeartbeatRequest_ConsumerData) isHeartbeatRequest_ClientData() {}
-
-func (m *HeartbeatRequest) GetClientData() isHeartbeatRequest_ClientData {
-	if m != nil {
-		return m.ClientData
-	}
-	return nil
-}
-
-func (m *HeartbeatRequest) GetProducerData() *ProducerData {
-	if x, ok := m.GetClientData().(*HeartbeatRequest_ProducerData); ok {
-		return x.ProducerData
-	}
-	return nil
-}
-
-func (m *HeartbeatRequest) GetConsumerData() *ConsumerData {
-	if x, ok := m.GetClientData().(*HeartbeatRequest_ConsumerData); ok {
-		return x.ConsumerData
-	}
-	return nil
-}
-
-func (m *HeartbeatRequest) GetFifoFlag() bool {
-	if m != nil {
-		return m.FifoFlag
-	}
-	return false
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*HeartbeatRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*HeartbeatRequest_ProducerData)(nil),
-		(*HeartbeatRequest_ConsumerData)(nil),
-	}
-}
-
-type HeartbeatResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *HeartbeatResponse) Reset()         { *m = HeartbeatResponse{} }
-func (m *HeartbeatResponse) String() string { return proto.CompactTextString(m) }
-func (*HeartbeatResponse) ProtoMessage()    {}
-func (*HeartbeatResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{16}
-}
-
-func (m *HeartbeatResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_HeartbeatResponse.Unmarshal(m, b)
-}
-func (m *HeartbeatResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_HeartbeatResponse.Marshal(b, m, deterministic)
-}
-func (m *HeartbeatResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_HeartbeatResponse.Merge(m, src)
-}
-func (m *HeartbeatResponse) XXX_Size() int {
-	return xxx_messageInfo_HeartbeatResponse.Size(m)
-}
-func (m *HeartbeatResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_HeartbeatResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_HeartbeatResponse proto.InternalMessageInfo
-
-func (m *HeartbeatResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type HealthCheckRequest struct {
-	Group                *Resource `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	ClientHost           string    `protobuf:"bytes,2,opt,name=client_host,json=clientHost,proto3" json:"client_host,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *HealthCheckRequest) Reset()         { *m = HealthCheckRequest{} }
-func (m *HealthCheckRequest) String() string { return proto.CompactTextString(m) }
-func (*HealthCheckRequest) ProtoMessage()    {}
-func (*HealthCheckRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{17}
-}
-
-func (m *HealthCheckRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_HealthCheckRequest.Unmarshal(m, b)
-}
-func (m *HealthCheckRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_HealthCheckRequest.Marshal(b, m, deterministic)
-}
-func (m *HealthCheckRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_HealthCheckRequest.Merge(m, src)
-}
-func (m *HealthCheckRequest) XXX_Size() int {
-	return xxx_messageInfo_HealthCheckRequest.Size(m)
-}
-func (m *HealthCheckRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_HealthCheckRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_HealthCheckRequest proto.InternalMessageInfo
-
-func (m *HealthCheckRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *HealthCheckRequest) GetClientHost() string {
-	if m != nil {
-		return m.ClientHost
-	}
-	return ""
-}
-
-type HealthCheckResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *HealthCheckResponse) Reset()         { *m = HealthCheckResponse{} }
-func (m *HealthCheckResponse) String() string { return proto.CompactTextString(m) }
-func (*HealthCheckResponse) ProtoMessage()    {}
-func (*HealthCheckResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{18}
-}
-
-func (m *HealthCheckResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_HealthCheckResponse.Unmarshal(m, b)
-}
-func (m *HealthCheckResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_HealthCheckResponse.Marshal(b, m, deterministic)
-}
-func (m *HealthCheckResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_HealthCheckResponse.Merge(m, src)
-}
-func (m *HealthCheckResponse) XXX_Size() int {
-	return xxx_messageInfo_HealthCheckResponse.Size(m)
-}
-func (m *HealthCheckResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_HealthCheckResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_HealthCheckResponse proto.InternalMessageInfo
-
-func (m *HealthCheckResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type EndTransactionRequest struct {
-	Group                *Resource                                   `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	MessageId            string                                      `protobuf:"bytes,2,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	TransactionId        string                                      `protobuf:"bytes,3,opt,name=transaction_id,json=transactionId,proto3" json:"transaction_id,omitempty"`
-	Resolution           EndTransactionRequest_TransactionResolution `protobuf:"varint,4,opt,name=resolution,proto3,enum=apache.rocketmq.v1.EndTransactionRequest_TransactionResolution" json:"resolution,omitempty"`
-	Source               EndTransactionRequest_Source                `protobuf:"varint,5,opt,name=source,proto3,enum=apache.rocketmq.v1.EndTransactionRequest_Source" json:"source,omitempty"`
-	TraceContext         string                                      `protobuf:"bytes,6,opt,name=trace_context,json=traceContext,proto3" json:"trace_context,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                    `json:"-"`
-	XXX_unrecognized     []byte                                      `json:"-"`
-	XXX_sizecache        int32                                       `json:"-"`
-}
-
-func (m *EndTransactionRequest) Reset()         { *m = EndTransactionRequest{} }
-func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) }
-func (*EndTransactionRequest) ProtoMessage()    {}
-func (*EndTransactionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{19}
-}
-
-func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EndTransactionRequest.Unmarshal(m, b)
-}
-func (m *EndTransactionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EndTransactionRequest.Marshal(b, m, deterministic)
-}
-func (m *EndTransactionRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EndTransactionRequest.Merge(m, src)
-}
-func (m *EndTransactionRequest) XXX_Size() int {
-	return xxx_messageInfo_EndTransactionRequest.Size(m)
-}
-func (m *EndTransactionRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_EndTransactionRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EndTransactionRequest proto.InternalMessageInfo
-
-func (m *EndTransactionRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *EndTransactionRequest) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
-	}
-	return ""
-}
-
-func (m *EndTransactionRequest) GetTransactionId() string {
-	if m != nil {
-		return m.TransactionId
-	}
-	return ""
-}
-
-func (m *EndTransactionRequest) GetResolution() EndTransactionRequest_TransactionResolution {
-	if m != nil {
-		return m.Resolution
-	}
-	return EndTransactionRequest_COMMIT
-}
-
-func (m *EndTransactionRequest) GetSource() EndTransactionRequest_Source {
-	if m != nil {
-		return m.Source
-	}
-	return EndTransactionRequest_CLIENT
-}
-
-func (m *EndTransactionRequest) GetTraceContext() string {
-	if m != nil {
-		return m.TraceContext
-	}
-	return ""
-}
-
-type EndTransactionResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *EndTransactionResponse) Reset()         { *m = EndTransactionResponse{} }
-func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) }
-func (*EndTransactionResponse) ProtoMessage()    {}
-func (*EndTransactionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{20}
-}
-
-func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EndTransactionResponse.Unmarshal(m, b)
-}
-func (m *EndTransactionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EndTransactionResponse.Marshal(b, m, deterministic)
-}
-func (m *EndTransactionResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EndTransactionResponse.Merge(m, src)
-}
-func (m *EndTransactionResponse) XXX_Size() int {
-	return xxx_messageInfo_EndTransactionResponse.Size(m)
-}
-func (m *EndTransactionResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_EndTransactionResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EndTransactionResponse proto.InternalMessageInfo
-
-func (m *EndTransactionResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type QueryOffsetRequest struct {
-	Partition            *Partition             `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"`
-	Policy               QueryOffsetPolicy      `protobuf:"varint,2,opt,name=policy,proto3,enum=apache.rocketmq.v1.QueryOffsetPolicy" json:"policy,omitempty"`
-	TimePoint            *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=time_point,json=timePoint,proto3" json:"time_point,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
-}
-
-func (m *QueryOffsetRequest) Reset()         { *m = QueryOffsetRequest{} }
-func (m *QueryOffsetRequest) String() string { return proto.CompactTextString(m) }
-func (*QueryOffsetRequest) ProtoMessage()    {}
-func (*QueryOffsetRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{21}
-}
-
-func (m *QueryOffsetRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryOffsetRequest.Unmarshal(m, b)
-}
-func (m *QueryOffsetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryOffsetRequest.Marshal(b, m, deterministic)
-}
-func (m *QueryOffsetRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryOffsetRequest.Merge(m, src)
-}
-func (m *QueryOffsetRequest) XXX_Size() int {
-	return xxx_messageInfo_QueryOffsetRequest.Size(m)
-}
-func (m *QueryOffsetRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryOffsetRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryOffsetRequest proto.InternalMessageInfo
-
-func (m *QueryOffsetRequest) GetPartition() *Partition {
-	if m != nil {
-		return m.Partition
-	}
-	return nil
-}
-
-func (m *QueryOffsetRequest) GetPolicy() QueryOffsetPolicy {
-	if m != nil {
-		return m.Policy
-	}
-	return QueryOffsetPolicy_BEGINNING
-}
-
-func (m *QueryOffsetRequest) GetTimePoint() *timestamppb.Timestamp {
-	if m != nil {
-		return m.TimePoint
-	}
-	return nil
-}
-
-type QueryOffsetResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	Offset               int64           `protobuf:"varint,2,opt,name=offset,proto3" json:"offset,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *QueryOffsetResponse) Reset()         { *m = QueryOffsetResponse{} }
-func (m *QueryOffsetResponse) String() string { return proto.CompactTextString(m) }
-func (*QueryOffsetResponse) ProtoMessage()    {}
-func (*QueryOffsetResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{22}
-}
-
-func (m *QueryOffsetResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryOffsetResponse.Unmarshal(m, b)
-}
-func (m *QueryOffsetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryOffsetResponse.Marshal(b, m, deterministic)
-}
-func (m *QueryOffsetResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryOffsetResponse.Merge(m, src)
-}
-func (m *QueryOffsetResponse) XXX_Size() int {
-	return xxx_messageInfo_QueryOffsetResponse.Size(m)
-}
-func (m *QueryOffsetResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryOffsetResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryOffsetResponse proto.InternalMessageInfo
-
-func (m *QueryOffsetResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func (m *QueryOffsetResponse) GetOffset() int64 {
-	if m != nil {
-		return m.Offset
-	}
-	return 0
-}
-
-type PullMessageRequest struct {
-	Group                *Resource            `protobuf:"bytes,1,opt,name=group,proto3" json:"group,omitempty"`
-	Partition            *Partition           `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"`
-	Offset               int64                `protobuf:"varint,3,opt,name=offset,proto3" json:"offset,omitempty"`
-	BatchSize            int32                `protobuf:"varint,4,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"`
-	AwaitTime            *durationpb.Duration `protobuf:"bytes,5,opt,name=await_time,json=awaitTime,proto3" json:"await_time,omitempty"`
-	FilterExpression     *FilterExpression    `protobuf:"bytes,6,opt,name=filter_expression,json=filterExpression,proto3" json:"filter_expression,omitempty"`
-	ClientId             string               `protobuf:"bytes,7,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
-}
-
-func (m *PullMessageRequest) Reset()         { *m = PullMessageRequest{} }
-func (m *PullMessageRequest) String() string { return proto.CompactTextString(m) }
-func (*PullMessageRequest) ProtoMessage()    {}
-func (*PullMessageRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{23}
-}
-
-func (m *PullMessageRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PullMessageRequest.Unmarshal(m, b)
-}
-func (m *PullMessageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PullMessageRequest.Marshal(b, m, deterministic)
-}
-func (m *PullMessageRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PullMessageRequest.Merge(m, src)
-}
-func (m *PullMessageRequest) XXX_Size() int {
-	return xxx_messageInfo_PullMessageRequest.Size(m)
-}
-func (m *PullMessageRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_PullMessageRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PullMessageRequest proto.InternalMessageInfo
-
-func (m *PullMessageRequest) GetGroup() *Resource {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *PullMessageRequest) GetPartition() *Partition {
-	if m != nil {
-		return m.Partition
-	}
-	return nil
-}
-
-func (m *PullMessageRequest) GetOffset() int64 {
-	if m != nil {
-		return m.Offset
-	}
-	return 0
-}
-
-func (m *PullMessageRequest) GetBatchSize() int32 {
-	if m != nil {
-		return m.BatchSize
-	}
-	return 0
-}
-
-func (m *PullMessageRequest) GetAwaitTime() *durationpb.Duration {
-	if m != nil {
-		return m.AwaitTime
-	}
-	return nil
-}
-
-func (m *PullMessageRequest) GetFilterExpression() *FilterExpression {
-	if m != nil {
-		return m.FilterExpression
-	}
-	return nil
-}
-
-func (m *PullMessageRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-type PullMessageResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	MinOffset            int64           `protobuf:"varint,2,opt,name=min_offset,json=minOffset,proto3" json:"min_offset,omitempty"`
-	NextOffset           int64           `protobuf:"varint,3,opt,name=next_offset,json=nextOffset,proto3" json:"next_offset,omitempty"`
-	MaxOffset            int64           `protobuf:"varint,4,opt,name=max_offset,json=maxOffset,proto3" json:"max_offset,omitempty"`
-	Messages             []*Message      `protobuf:"bytes,5,rep,name=messages,proto3" json:"messages,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *PullMessageResponse) Reset()         { *m = PullMessageResponse{} }
-func (m *PullMessageResponse) String() string { return proto.CompactTextString(m) }
-func (*PullMessageResponse) ProtoMessage()    {}
-func (*PullMessageResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{24}
-}
-
-func (m *PullMessageResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PullMessageResponse.Unmarshal(m, b)
-}
-func (m *PullMessageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PullMessageResponse.Marshal(b, m, deterministic)
-}
-func (m *PullMessageResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PullMessageResponse.Merge(m, src)
-}
-func (m *PullMessageResponse) XXX_Size() int {
-	return xxx_messageInfo_PullMessageResponse.Size(m)
-}
-func (m *PullMessageResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_PullMessageResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PullMessageResponse proto.InternalMessageInfo
-
-func (m *PullMessageResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func (m *PullMessageResponse) GetMinOffset() int64 {
-	if m != nil {
-		return m.MinOffset
-	}
-	return 0
-}
-
-func (m *PullMessageResponse) GetNextOffset() int64 {
-	if m != nil {
-		return m.NextOffset
-	}
-	return 0
-}
-
-func (m *PullMessageResponse) GetMaxOffset() int64 {
-	if m != nil {
-		return m.MaxOffset
-	}
-	return 0
-}
-
-func (m *PullMessageResponse) GetMessages() []*Message {
-	if m != nil {
-		return m.Messages
-	}
-	return nil
-}
-
-type NoopCommand struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *NoopCommand) Reset()         { *m = NoopCommand{} }
-func (m *NoopCommand) String() string { return proto.CompactTextString(m) }
-func (*NoopCommand) ProtoMessage()    {}
-func (*NoopCommand) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{25}
-}
-
-func (m *NoopCommand) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_NoopCommand.Unmarshal(m, b)
-}
-func (m *NoopCommand) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_NoopCommand.Marshal(b, m, deterministic)
-}
-func (m *NoopCommand) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_NoopCommand.Merge(m, src)
-}
-func (m *NoopCommand) XXX_Size() int {
-	return xxx_messageInfo_NoopCommand.Size(m)
-}
-func (m *NoopCommand) XXX_DiscardUnknown() {
-	xxx_messageInfo_NoopCommand.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_NoopCommand proto.InternalMessageInfo
-
-type PrintThreadStackTraceCommand struct {
-	CommandId            string   `protobuf:"bytes,1,opt,name=command_id,json=commandId,proto3" json:"command_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PrintThreadStackTraceCommand) Reset()         { *m = PrintThreadStackTraceCommand{} }
-func (m *PrintThreadStackTraceCommand) String() string { return proto.CompactTextString(m) }
-func (*PrintThreadStackTraceCommand) ProtoMessage()    {}
-func (*PrintThreadStackTraceCommand) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{26}
-}
-
-func (m *PrintThreadStackTraceCommand) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PrintThreadStackTraceCommand.Unmarshal(m, b)
-}
-func (m *PrintThreadStackTraceCommand) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PrintThreadStackTraceCommand.Marshal(b, m, deterministic)
-}
-func (m *PrintThreadStackTraceCommand) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PrintThreadStackTraceCommand.Merge(m, src)
-}
-func (m *PrintThreadStackTraceCommand) XXX_Size() int {
-	return xxx_messageInfo_PrintThreadStackTraceCommand.Size(m)
-}
-func (m *PrintThreadStackTraceCommand) XXX_DiscardUnknown() {
-	xxx_messageInfo_PrintThreadStackTraceCommand.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PrintThreadStackTraceCommand proto.InternalMessageInfo
-
-func (m *PrintThreadStackTraceCommand) GetCommandId() string {
-	if m != nil {
-		return m.CommandId
-	}
-	return ""
-}
-
-type ReportThreadStackTraceRequest struct {
-	CommandId            string   `protobuf:"bytes,1,opt,name=command_id,json=commandId,proto3" json:"command_id,omitempty"`
-	ThreadStackTrace     string   `protobuf:"bytes,2,opt,name=thread_stack_trace,json=threadStackTrace,proto3" json:"thread_stack_trace,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ReportThreadStackTraceRequest) Reset()         { *m = ReportThreadStackTraceRequest{} }
-func (m *ReportThreadStackTraceRequest) String() string { return proto.CompactTextString(m) }
-func (*ReportThreadStackTraceRequest) ProtoMessage()    {}
-func (*ReportThreadStackTraceRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{27}
-}
-
-func (m *ReportThreadStackTraceRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReportThreadStackTraceRequest.Unmarshal(m, b)
-}
-func (m *ReportThreadStackTraceRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReportThreadStackTraceRequest.Marshal(b, m, deterministic)
-}
-func (m *ReportThreadStackTraceRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReportThreadStackTraceRequest.Merge(m, src)
-}
-func (m *ReportThreadStackTraceRequest) XXX_Size() int {
-	return xxx_messageInfo_ReportThreadStackTraceRequest.Size(m)
-}
-func (m *ReportThreadStackTraceRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReportThreadStackTraceRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReportThreadStackTraceRequest proto.InternalMessageInfo
-
-func (m *ReportThreadStackTraceRequest) GetCommandId() string {
-	if m != nil {
-		return m.CommandId
-	}
-	return ""
-}
-
-func (m *ReportThreadStackTraceRequest) GetThreadStackTrace() string {
-	if m != nil {
-		return m.ThreadStackTrace
-	}
-	return ""
-}
-
-type ReportThreadStackTraceResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *ReportThreadStackTraceResponse) Reset()         { *m = ReportThreadStackTraceResponse{} }
-func (m *ReportThreadStackTraceResponse) String() string { return proto.CompactTextString(m) }
-func (*ReportThreadStackTraceResponse) ProtoMessage()    {}
-func (*ReportThreadStackTraceResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{28}
-}
-
-func (m *ReportThreadStackTraceResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReportThreadStackTraceResponse.Unmarshal(m, b)
-}
-func (m *ReportThreadStackTraceResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReportThreadStackTraceResponse.Marshal(b, m, deterministic)
-}
-func (m *ReportThreadStackTraceResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReportThreadStackTraceResponse.Merge(m, src)
-}
-func (m *ReportThreadStackTraceResponse) XXX_Size() int {
-	return xxx_messageInfo_ReportThreadStackTraceResponse.Size(m)
-}
-func (m *ReportThreadStackTraceResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReportThreadStackTraceResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReportThreadStackTraceResponse proto.InternalMessageInfo
-
-func (m *ReportThreadStackTraceResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type VerifyMessageConsumptionCommand struct {
-	CommandId            string   `protobuf:"bytes,1,opt,name=command_id,json=commandId,proto3" json:"command_id,omitempty"`
-	Message              *Message `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *VerifyMessageConsumptionCommand) Reset()         { *m = VerifyMessageConsumptionCommand{} }
-func (m *VerifyMessageConsumptionCommand) String() string { return proto.CompactTextString(m) }
-func (*VerifyMessageConsumptionCommand) ProtoMessage()    {}
-func (*VerifyMessageConsumptionCommand) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{29}
-}
-
-func (m *VerifyMessageConsumptionCommand) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_VerifyMessageConsumptionCommand.Unmarshal(m, b)
-}
-func (m *VerifyMessageConsumptionCommand) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_VerifyMessageConsumptionCommand.Marshal(b, m, deterministic)
-}
-func (m *VerifyMessageConsumptionCommand) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_VerifyMessageConsumptionCommand.Merge(m, src)
-}
-func (m *VerifyMessageConsumptionCommand) XXX_Size() int {
-	return xxx_messageInfo_VerifyMessageConsumptionCommand.Size(m)
-}
-func (m *VerifyMessageConsumptionCommand) XXX_DiscardUnknown() {
-	xxx_messageInfo_VerifyMessageConsumptionCommand.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_VerifyMessageConsumptionCommand proto.InternalMessageInfo
-
-func (m *VerifyMessageConsumptionCommand) GetCommandId() string {
-	if m != nil {
-		return m.CommandId
-	}
-	return ""
-}
-
-func (m *VerifyMessageConsumptionCommand) GetMessage() *Message {
-	if m != nil {
-		return m.Message
-	}
-	return nil
-}
-
-type ReportMessageConsumptionResultRequest struct {
-	CommandId string `protobuf:"bytes,1,opt,name=command_id,json=commandId,proto3" json:"command_id,omitempty"`
-	// 1. Return `INVALID_ARGUMENT` if message is corrupted.
-	// 2. Return `INTERNAL` if failed to consume message.
-	// 3. Return `OK` if success.
-	Status               *status.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
-}
-
-func (m *ReportMessageConsumptionResultRequest) Reset()         { *m = ReportMessageConsumptionResultRequest{} }
-func (m *ReportMessageConsumptionResultRequest) String() string { return proto.CompactTextString(m) }
-func (*ReportMessageConsumptionResultRequest) ProtoMessage()    {}
-func (*ReportMessageConsumptionResultRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{30}
-}
-
-func (m *ReportMessageConsumptionResultRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReportMessageConsumptionResultRequest.Unmarshal(m, b)
-}
-func (m *ReportMessageConsumptionResultRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReportMessageConsumptionResultRequest.Marshal(b, m, deterministic)
-}
-func (m *ReportMessageConsumptionResultRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReportMessageConsumptionResultRequest.Merge(m, src)
-}
-func (m *ReportMessageConsumptionResultRequest) XXX_Size() int {
-	return xxx_messageInfo_ReportMessageConsumptionResultRequest.Size(m)
-}
-func (m *ReportMessageConsumptionResultRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReportMessageConsumptionResultRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReportMessageConsumptionResultRequest proto.InternalMessageInfo
-
-func (m *ReportMessageConsumptionResultRequest) GetCommandId() string {
-	if m != nil {
-		return m.CommandId
-	}
-	return ""
-}
-
-func (m *ReportMessageConsumptionResultRequest) GetStatus() *status.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-type ReportMessageConsumptionResultResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *ReportMessageConsumptionResultResponse) Reset() {
-	*m = ReportMessageConsumptionResultResponse{}
-}
-func (m *ReportMessageConsumptionResultResponse) String() string { return proto.CompactTextString(m) }
-func (*ReportMessageConsumptionResultResponse) ProtoMessage()    {}
-func (*ReportMessageConsumptionResultResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{31}
-}
-
-func (m *ReportMessageConsumptionResultResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReportMessageConsumptionResultResponse.Unmarshal(m, b)
-}
-func (m *ReportMessageConsumptionResultResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReportMessageConsumptionResultResponse.Marshal(b, m, deterministic)
-}
-func (m *ReportMessageConsumptionResultResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReportMessageConsumptionResultResponse.Merge(m, src)
-}
-func (m *ReportMessageConsumptionResultResponse) XXX_Size() int {
-	return xxx_messageInfo_ReportMessageConsumptionResultResponse.Size(m)
-}
-func (m *ReportMessageConsumptionResultResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReportMessageConsumptionResultResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReportMessageConsumptionResultResponse proto.InternalMessageInfo
-
-func (m *ReportMessageConsumptionResultResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-type RecoverOrphanedTransactionCommand struct {
-	OrphanedTransactionalMessage *Message `protobuf:"bytes,1,opt,name=orphaned_transactional_message,json=orphanedTransactionalMessage,proto3" json:"orphaned_transactional_message,omitempty"`
-	TransactionId                string   `protobuf:"bytes,2,opt,name=transaction_id,json=transactionId,proto3" json:"transaction_id,omitempty"`
-	XXX_NoUnkeyedLiteral         struct{} `json:"-"`
-	XXX_unrecognized             []byte   `json:"-"`
-	XXX_sizecache                int32    `json:"-"`
-}
-
-func (m *RecoverOrphanedTransactionCommand) Reset()         { *m = RecoverOrphanedTransactionCommand{} }
-func (m *RecoverOrphanedTransactionCommand) String() string { return proto.CompactTextString(m) }
-func (*RecoverOrphanedTransactionCommand) ProtoMessage()    {}
-func (*RecoverOrphanedTransactionCommand) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{32}
-}
-
-func (m *RecoverOrphanedTransactionCommand) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RecoverOrphanedTransactionCommand.Unmarshal(m, b)
-}
-func (m *RecoverOrphanedTransactionCommand) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RecoverOrphanedTransactionCommand.Marshal(b, m, deterministic)
-}
-func (m *RecoverOrphanedTransactionCommand) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RecoverOrphanedTransactionCommand.Merge(m, src)
-}
-func (m *RecoverOrphanedTransactionCommand) XXX_Size() int {
-	return xxx_messageInfo_RecoverOrphanedTransactionCommand.Size(m)
-}
-func (m *RecoverOrphanedTransactionCommand) XXX_DiscardUnknown() {
-	xxx_messageInfo_RecoverOrphanedTransactionCommand.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RecoverOrphanedTransactionCommand proto.InternalMessageInfo
-
-func (m *RecoverOrphanedTransactionCommand) GetOrphanedTransactionalMessage() *Message {
-	if m != nil {
-		return m.OrphanedTransactionalMessage
-	}
-	return nil
-}
-
-func (m *RecoverOrphanedTransactionCommand) GetTransactionId() string {
-	if m != nil {
-		return m.TransactionId
-	}
-	return ""
-}
-
-type PollCommandRequest struct {
-	ClientId string      `protobuf:"bytes,1,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	Topics   []*Resource `protobuf:"bytes,2,rep,name=topics,proto3" json:"topics,omitempty"`
-	// Types that are valid to be assigned to Group:
-	//	*PollCommandRequest_ProducerGroup
-	//	*PollCommandRequest_ConsumerGroup
-	Group                isPollCommandRequest_Group `protobuf_oneof:"group"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
-}
-
-func (m *PollCommandRequest) Reset()         { *m = PollCommandRequest{} }
-func (m *PollCommandRequest) String() string { return proto.CompactTextString(m) }
-func (*PollCommandRequest) ProtoMessage()    {}
-func (*PollCommandRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{33}
-}
-
-func (m *PollCommandRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PollCommandRequest.Unmarshal(m, b)
-}
-func (m *PollCommandRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PollCommandRequest.Marshal(b, m, deterministic)
-}
-func (m *PollCommandRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PollCommandRequest.Merge(m, src)
-}
-func (m *PollCommandRequest) XXX_Size() int {
-	return xxx_messageInfo_PollCommandRequest.Size(m)
-}
-func (m *PollCommandRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_PollCommandRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PollCommandRequest proto.InternalMessageInfo
-
-func (m *PollCommandRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-func (m *PollCommandRequest) GetTopics() []*Resource {
-	if m != nil {
-		return m.Topics
-	}
-	return nil
-}
-
-type isPollCommandRequest_Group interface {
-	isPollCommandRequest_Group()
-}
-
-type PollCommandRequest_ProducerGroup struct {
-	ProducerGroup *Resource `protobuf:"bytes,3,opt,name=producer_group,json=producerGroup,proto3,oneof"`
-}
-
-type PollCommandRequest_ConsumerGroup struct {
-	ConsumerGroup *Resource `protobuf:"bytes,4,opt,name=consumer_group,json=consumerGroup,proto3,oneof"`
-}
-
-func (*PollCommandRequest_ProducerGroup) isPollCommandRequest_Group() {}
-
-func (*PollCommandRequest_ConsumerGroup) isPollCommandRequest_Group() {}
-
-func (m *PollCommandRequest) GetGroup() isPollCommandRequest_Group {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *PollCommandRequest) GetProducerGroup() *Resource {
-	if x, ok := m.GetGroup().(*PollCommandRequest_ProducerGroup); ok {
-		return x.ProducerGroup
-	}
-	return nil
-}
-
-func (m *PollCommandRequest) GetConsumerGroup() *Resource {
-	if x, ok := m.GetGroup().(*PollCommandRequest_ConsumerGroup); ok {
-		return x.ConsumerGroup
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*PollCommandRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*PollCommandRequest_ProducerGroup)(nil),
-		(*PollCommandRequest_ConsumerGroup)(nil),
-	}
-}
-
-type PollCommandResponse struct {
-	// Types that are valid to be assigned to Type:
-	//	*PollCommandResponse_NoopCommand
-	//	*PollCommandResponse_PrintThreadStackTraceCommand
-	//	*PollCommandResponse_VerifyMessageConsumptionCommand
-	//	*PollCommandResponse_RecoverOrphanedTransactionCommand
-	Type                 isPollCommandResponse_Type `protobuf_oneof:"type"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
-}
-
-func (m *PollCommandResponse) Reset()         { *m = PollCommandResponse{} }
-func (m *PollCommandResponse) String() string { return proto.CompactTextString(m) }
-func (*PollCommandResponse) ProtoMessage()    {}
-func (*PollCommandResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{34}
-}
-
-func (m *PollCommandResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PollCommandResponse.Unmarshal(m, b)
-}
-func (m *PollCommandResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PollCommandResponse.Marshal(b, m, deterministic)
-}
-func (m *PollCommandResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PollCommandResponse.Merge(m, src)
-}
-func (m *PollCommandResponse) XXX_Size() int {
-	return xxx_messageInfo_PollCommandResponse.Size(m)
-}
-func (m *PollCommandResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_PollCommandResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PollCommandResponse proto.InternalMessageInfo
-
-type isPollCommandResponse_Type interface {
-	isPollCommandResponse_Type()
-}
-
-type PollCommandResponse_NoopCommand struct {
-	NoopCommand *NoopCommand `protobuf:"bytes,1,opt,name=noop_command,json=noopCommand,proto3,oneof"`
-}
-
-type PollCommandResponse_PrintThreadStackTraceCommand struct {
-	PrintThreadStackTraceCommand *PrintThreadStackTraceCommand `protobuf:"bytes,2,opt,name=print_thread_stack_trace_command,json=printThreadStackTraceCommand,proto3,oneof"`
-}
-
-type PollCommandResponse_VerifyMessageConsumptionCommand struct {
-	VerifyMessageConsumptionCommand *VerifyMessageConsumptionCommand `protobuf:"bytes,3,opt,name=verify_message_consumption_command,json=verifyMessageConsumptionCommand,proto3,oneof"`
-}
-
-type PollCommandResponse_RecoverOrphanedTransactionCommand struct {
-	RecoverOrphanedTransactionCommand *RecoverOrphanedTransactionCommand `protobuf:"bytes,4,opt,name=recover_orphaned_transaction_command,json=recoverOrphanedTransactionCommand,proto3,oneof"`
-}
-
-func (*PollCommandResponse_NoopCommand) isPollCommandResponse_Type() {}
-
-func (*PollCommandResponse_PrintThreadStackTraceCommand) isPollCommandResponse_Type() {}
-
-func (*PollCommandResponse_VerifyMessageConsumptionCommand) isPollCommandResponse_Type() {}
-
-func (*PollCommandResponse_RecoverOrphanedTransactionCommand) isPollCommandResponse_Type() {}
-
-func (m *PollCommandResponse) GetType() isPollCommandResponse_Type {
-	if m != nil {
-		return m.Type
-	}
-	return nil
-}
-
-func (m *PollCommandResponse) GetNoopCommand() *NoopCommand {
-	if x, ok := m.GetType().(*PollCommandResponse_NoopCommand); ok {
-		return x.NoopCommand
-	}
-	return nil
-}
-
-func (m *PollCommandResponse) GetPrintThreadStackTraceCommand() *PrintThreadStackTraceCommand {
-	if x, ok := m.GetType().(*PollCommandResponse_PrintThreadStackTraceCommand); ok {
-		return x.PrintThreadStackTraceCommand
-	}
-	return nil
-}
-
-func (m *PollCommandResponse) GetVerifyMessageConsumptionCommand() *VerifyMessageConsumptionCommand {
-	if x, ok := m.GetType().(*PollCommandResponse_VerifyMessageConsumptionCommand); ok {
-		return x.VerifyMessageConsumptionCommand
-	}
-	return nil
-}
-
-func (m *PollCommandResponse) GetRecoverOrphanedTransactionCommand() *RecoverOrphanedTransactionCommand {
-	if x, ok := m.GetType().(*PollCommandResponse_RecoverOrphanedTransactionCommand); ok {
-		return x.RecoverOrphanedTransactionCommand
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*PollCommandResponse) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*PollCommandResponse_NoopCommand)(nil),
-		(*PollCommandResponse_PrintThreadStackTraceCommand)(nil),
-		(*PollCommandResponse_VerifyMessageConsumptionCommand)(nil),
-		(*PollCommandResponse_RecoverOrphanedTransactionCommand)(nil),
-	}
-}
-
-type NotifyClientTerminationRequest struct {
-	// Types that are valid to be assigned to Group:
-	//	*NotifyClientTerminationRequest_ProducerGroup
-	//	*NotifyClientTerminationRequest_ConsumerGroup
-	Group                isNotifyClientTerminationRequest_Group `protobuf_oneof:"group"`
-	ClientId             string                                 `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                               `json:"-"`
-	XXX_unrecognized     []byte                                 `json:"-"`
-	XXX_sizecache        int32                                  `json:"-"`
-}
-
-func (m *NotifyClientTerminationRequest) Reset()         { *m = NotifyClientTerminationRequest{} }
-func (m *NotifyClientTerminationRequest) String() string { return proto.CompactTextString(m) }
-func (*NotifyClientTerminationRequest) ProtoMessage()    {}
-func (*NotifyClientTerminationRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{35}
-}
-
-func (m *NotifyClientTerminationRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_NotifyClientTerminationRequest.Unmarshal(m, b)
-}
-func (m *NotifyClientTerminationRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_NotifyClientTerminationRequest.Marshal(b, m, deterministic)
-}
-func (m *NotifyClientTerminationRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_NotifyClientTerminationRequest.Merge(m, src)
-}
-func (m *NotifyClientTerminationRequest) XXX_Size() int {
-	return xxx_messageInfo_NotifyClientTerminationRequest.Size(m)
-}
-func (m *NotifyClientTerminationRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_NotifyClientTerminationRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_NotifyClientTerminationRequest proto.InternalMessageInfo
-
-type isNotifyClientTerminationRequest_Group interface {
-	isNotifyClientTerminationRequest_Group()
-}
-
-type NotifyClientTerminationRequest_ProducerGroup struct {
-	ProducerGroup *Resource `protobuf:"bytes,1,opt,name=producer_group,json=producerGroup,proto3,oneof"`
-}
-
-type NotifyClientTerminationRequest_ConsumerGroup struct {
-	ConsumerGroup *Resource `protobuf:"bytes,2,opt,name=consumer_group,json=consumerGroup,proto3,oneof"`
-}
-
-func (*NotifyClientTerminationRequest_ProducerGroup) isNotifyClientTerminationRequest_Group() {}
-
-func (*NotifyClientTerminationRequest_ConsumerGroup) isNotifyClientTerminationRequest_Group() {}
-
-func (m *NotifyClientTerminationRequest) GetGroup() isNotifyClientTerminationRequest_Group {
-	if m != nil {
-		return m.Group
-	}
-	return nil
-}
-
-func (m *NotifyClientTerminationRequest) GetProducerGroup() *Resource {
-	if x, ok := m.GetGroup().(*NotifyClientTerminationRequest_ProducerGroup); ok {
-		return x.ProducerGroup
-	}
-	return nil
-}
-
-func (m *NotifyClientTerminationRequest) GetConsumerGroup() *Resource {
-	if x, ok := m.GetGroup().(*NotifyClientTerminationRequest_ConsumerGroup); ok {
-		return x.ConsumerGroup
-	}
-	return nil
-}
-
-func (m *NotifyClientTerminationRequest) GetClientId() string {
-	if m != nil {
-		return m.ClientId
-	}
-	return ""
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*NotifyClientTerminationRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*NotifyClientTerminationRequest_ProducerGroup)(nil),
-		(*NotifyClientTerminationRequest_ConsumerGroup)(nil),
-	}
-}
-
-type NotifyClientTerminationResponse struct {
-	Common               *ResponseCommon `protobuf:"bytes,1,opt,name=common,proto3" json:"common,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *NotifyClientTerminationResponse) Reset()         { *m = NotifyClientTerminationResponse{} }
-func (m *NotifyClientTerminationResponse) String() string { return proto.CompactTextString(m) }
-func (*NotifyClientTerminationResponse) ProtoMessage()    {}
-func (*NotifyClientTerminationResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_ebe5e0b089e977b9, []int{36}
-}
-
-func (m *NotifyClientTerminationResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_NotifyClientTerminationResponse.Unmarshal(m, b)
-}
-func (m *NotifyClientTerminationResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_NotifyClientTerminationResponse.Marshal(b, m, deterministic)
-}
-func (m *NotifyClientTerminationResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_NotifyClientTerminationResponse.Merge(m, src)
-}
-func (m *NotifyClientTerminationResponse) XXX_Size() int {
-	return xxx_messageInfo_NotifyClientTerminationResponse.Size(m)
-}
-func (m *NotifyClientTerminationResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_NotifyClientTerminationResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_NotifyClientTerminationResponse proto.InternalMessageInfo
-
-func (m *NotifyClientTerminationResponse) GetCommon() *ResponseCommon {
-	if m != nil {
-		return m.Common
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("apache.rocketmq.v1.EndTransactionRequest_TransactionResolution", EndTransactionRequest_TransactionResolution_name, EndTransactionRequest_TransactionResolution_value)
-	proto.RegisterEnum("apache.rocketmq.v1.EndTransactionRequest_Source", EndTransactionRequest_Source_name, EndTransactionRequest_Source_value)
-	proto.RegisterType((*ResponseCommon)(nil), "apache.rocketmq.v1.ResponseCommon")
-	proto.RegisterType((*QueryRouteRequest)(nil), "apache.rocketmq.v1.QueryRouteRequest")
-	proto.RegisterType((*QueryRouteResponse)(nil), "apache.rocketmq.v1.QueryRouteResponse")
-	proto.RegisterType((*SendMessageRequest)(nil), "apache.rocketmq.v1.SendMessageRequest")
-	proto.RegisterType((*SendMessageResponse)(nil), "apache.rocketmq.v1.SendMessageResponse")
-	proto.RegisterType((*QueryAssignmentRequest)(nil), "apache.rocketmq.v1.QueryAssignmentRequest")
-	proto.RegisterType((*QueryAssignmentResponse)(nil), "apache.rocketmq.v1.QueryAssignmentResponse")
-	proto.RegisterType((*ReceiveMessageRequest)(nil), "apache.rocketmq.v1.ReceiveMessageRequest")
-	proto.RegisterType((*ReceiveMessageResponse)(nil), "apache.rocketmq.v1.ReceiveMessageResponse")
-	proto.RegisterType((*AckMessageRequest)(nil), "apache.rocketmq.v1.AckMessageRequest")
-	proto.RegisterType((*AckMessageResponse)(nil), "apache.rocketmq.v1.AckMessageResponse")
-	proto.RegisterType((*NackMessageRequest)(nil), "apache.rocketmq.v1.NackMessageRequest")
-	proto.RegisterType((*NackMessageResponse)(nil), "apache.rocketmq.v1.NackMessageResponse")
-	proto.RegisterType((*ForwardMessageToDeadLetterQueueRequest)(nil), "apache.rocketmq.v1.ForwardMessageToDeadLetterQueueRequest")
-	proto.RegisterType((*ForwardMessageToDeadLetterQueueResponse)(nil), "apache.rocketmq.v1.ForwardMessageToDeadLetterQueueResponse")
-	proto.RegisterType((*HeartbeatRequest)(nil), "apache.rocketmq.v1.HeartbeatRequest")
-	proto.RegisterType((*HeartbeatResponse)(nil), "apache.rocketmq.v1.HeartbeatResponse")
-	proto.RegisterType((*HealthCheckRequest)(nil), "apache.rocketmq.v1.HealthCheckRequest")
-	proto.RegisterType((*HealthCheckResponse)(nil), "apache.rocketmq.v1.HealthCheckResponse")
-	proto.RegisterType((*EndTransactionRequest)(nil), "apache.rocketmq.v1.EndTransactionRequest")
-	proto.RegisterType((*EndTransactionResponse)(nil), "apache.rocketmq.v1.EndTransactionResponse")
-	proto.RegisterType((*QueryOffsetRequest)(nil), "apache.rocketmq.v1.QueryOffsetRequest")
-	proto.RegisterType((*QueryOffsetResponse)(nil), "apache.rocketmq.v1.QueryOffsetResponse")
-	proto.RegisterType((*PullMessageRequest)(nil), "apache.rocketmq.v1.PullMessageRequest")
-	proto.RegisterType((*PullMessageResponse)(nil), "apache.rocketmq.v1.PullMessageResponse")
-	proto.RegisterType((*NoopCommand)(nil), "apache.rocketmq.v1.NoopCommand")
-	proto.RegisterType((*PrintThreadStackTraceCommand)(nil), "apache.rocketmq.v1.PrintThreadStackTraceCommand")
-	proto.RegisterType((*ReportThreadStackTraceRequest)(nil), "apache.rocketmq.v1.ReportThreadStackTraceRequest")
-	proto.RegisterType((*ReportThreadStackTraceResponse)(nil), "apache.rocketmq.v1.ReportThreadStackTraceResponse")
-	proto.RegisterType((*VerifyMessageConsumptionCommand)(nil), "apache.rocketmq.v1.VerifyMessageConsumptionCommand")
-	proto.RegisterType((*ReportMessageConsumptionResultRequest)(nil), "apache.rocketmq.v1.ReportMessageConsumptionResultRequest")
-	proto.RegisterType((*ReportMessageConsumptionResultResponse)(nil), "apache.rocketmq.v1.ReportMessageConsumptionResultResponse")
-	proto.RegisterType((*RecoverOrphanedTransactionCommand)(nil), "apache.rocketmq.v1.RecoverOrphanedTransactionCommand")
-	proto.RegisterType((*PollCommandRequest)(nil), "apache.rocketmq.v1.PollCommandRequest")
-	proto.RegisterType((*PollCommandResponse)(nil), "apache.rocketmq.v1.PollCommandResponse")
-	proto.RegisterType((*NotifyClientTerminationRequest)(nil), "apache.rocketmq.v1.NotifyClientTerminationRequest")
-	proto.RegisterType((*NotifyClientTerminationResponse)(nil), "apache.rocketmq.v1.NotifyClientTerminationResponse")
-}
-
-func init() { proto.RegisterFile("apache/rocketmq/v1/service.proto", fileDescriptor_ebe5e0b089e977b9) }
-
-var fileDescriptor_ebe5e0b089e977b9 = []byte{
-	// 2275 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x1a, 0x4d, 0x6f, 0x1b, 0xc7,
-	0x95, 0x4b, 0x52, 0x34, 0xf9, 0x24, 0x2b, 0xd4, 0x08, 0xb6, 0x55, 0xc6, 0xb6, 0x64, 0xc6, 0x96,
-	0x1d, 0xb7, 0x90, 0x62, 0x39, 0x41, 0x1b, 0xa7, 0x41, 0x4b, 0x7d, 0x38, 0x94, 0xea, 0x0f, 0x79,
-	0xa5, 0x38, 0x40, 0x80, 0x60, 0x3b, 0x5a, 0x0e, 0xa5, 0xad, 0x97, 0x3b, 0x9b, 0xd9, 0xa1, 0x22,
-	0x19, 0xe8, 0xa1, 0x41, 0x0e, 0x45, 0x0e, 0xbd, 0xf5, 0x50, 0xa0, 0x05, 0x7a, 0x2c, 0x7a, 0xed,
-	0x3f, 0x28, 0x8a, 0xde, 0x7b, 0x0b, 0x7a, 0x69, 0x7f, 0x40, 0x81, 0x1e, 0x8a, 0xde, 0x8b, 0x9d,
-	0x99, 0x5d, 0xee, 0x72, 0x87, 0x1f, 0x92, 0x78, 0xe8, 0x21, 0x37, 0xee, 0xcc, 0xfb, 0x9a, 0xf7,
-	0xde, 0xbc, 0xaf, 0x21, 0x2c, 0x61, 0x1f, 0xdb, 0x47, 0x64, 0x95, 0x51, 0xfb, 0x15, 0xe1, 0x9d,
-	0xcf, 0x57, 0x8f, 0x1f, 0xac, 0x06, 0x84, 0x1d, 0x3b, 0x36, 0x59, 0xf1, 0x19, 0xe5, 0x14, 0x21,
-	0x09, 0xb1, 0x12, 0x41, 0xac, 0x1c, 0x3f, 0xa8, 0xdd, 0x3c, 0xa4, 0xf4, 0xd0, 0x25, 0xab, 0x02,
-	0xe2, 0xa0, 0xdb, 0x5e, 0x6d, 0x75, 0x19, 0xe6, 0x0e, 0xf5, 0x24, 0x4e, 0x6d, 0xb1, 0x7f, 0x9f,
-	0x3b, 0x1d, 0x12, 0x70, 0xdc, 0xf1, 0x15, 0x40, 0x44, 0x80, 0xf9, 0xf6, 0x2a, 0x61, 0x8c, 0x32,
-	0xab, 0x45, 0x38, 0x76, 0xdc, 0x40, 0xed, 0x5f, 0x4b, 0xec, 0x07, 0x1c, 0xf3, 0x6e, 0xb4, 0xf1,
-	0x96, 0x46, 0xde, 0x16, 0x69, 0x3b, 0x9e, 0xd3, 0x63, 0x5f, 0xff, 0x6b, 0x1e, 0x66, 0x4d, 0x12,
-	0xf8, 0xd4, 0x0b, 0xc8, 0x06, 0xed, 0x74, 0xa8, 0x87, 0xee, 0x43, 0x49, 0xd2, 0x59, 0x30, 0x96,
-	0x8c, 0x7b, 0xd3, 0x6b, 0x68, 0x45, 0x72, 0x58, 0x61, 0xbe, 0xbd, 0xb2, 0x27, 0x76, 0x4c, 0x05,
-	0x81, 0x1e, 0xc1, 0x0c, 0x23, 0x9f, 0x77, 0x49, 0xc0, 0x2d, 0xc7, 0x6b, 0xd3, 0x85, 0xbc, 0xc0,
-	0xb8, 0x96, 0xc4, 0x30, 0xe5, 0xfe, 0xb6, 0xd7, 0xa6, 0xe6, 0x34, 0xeb, 0x7d, 0xa0, 0xdb, 0x50,
-	0x3c, 0x22, 0xae, 0xbf, 0x50, 0x10, 0x38, 0xd5, 0x24, 0x4e, 0x93, 0xb8, 0xbe, 0x29, 0x76, 0xd1,
-	0xbb, 0x00, 0x8c, 0x70, 0x76, 0x2a, 0xe9, 0x17, 0x05, 0xec, 0x95, 0x34, 0x7d, 0xce, 0x4e, 0x05,
-	0xf5, 0x0a, 0x8b, 0x7e, 0x86, 0x58, 0x2d, 0x72, 0xd0, 0x3d, 0x94, 0x58, 0x53, 0x59, 0xac, 0xcd,
-	0x70, 0x57, 0x62, 0xb5, 0xa2, 0x9f, 0x21, 0x96, 0xd4, 0xb0, 0xc0, 0x2a, 0x65, 0xb1, 0xb6, 0xc2,
-	0x5d, 0x89, 0x45, 0xa2, 0x9f, 0x3b, 0xc5, 0xf2, 0xa5, 0x6a, 0xa3, 0xfe, 0xb5, 0x01, 0x73, 0x2f,
-	0xba, 0x84, 0x9d, 0x9a, 0xb4, 0xcb, 0x89, 0x3a, 0x34, 0x5a, 0x83, 0x29, 0x4e, 0x7d, 0xc7, 0x56,
-	0xaa, 0xbc, 0xbe, 0x92, 0xf5, 0x90, 0x15, 0x93, 0x04, 0xb4, 0xcb, 0x6c, 0x62, 0x4a, 0x50, 0xf4,
-	0x01, 0x54, 0x88, 0xd7, 0xf2, 0xa9, 0xe3, 0xf1, 0x40, 0x29, 0xf4, 0x86, 0x0e, 0x6f, 0x2b, 0x02,
-	0x32, 0x7b, 0xf0, 0x3b, 0xc5, 0x72, 0xa1, 0xda, 0xa8, 0xff, 0xda, 0x00, 0x94, 0x14, 0x46, 0xda,
-	0x17, 0x3d, 0x82, 0x92, 0x2d, 0x6c, 0xac, 0xc4, 0xa9, 0x0f, 0x10, 0x27, 0xe1, 0x0d, 0xa6, 0xc2,
-	0x40, 0x1f, 0x02, 0xf8, 0x98, 0x71, 0xe1, 0x3b, 0xa1, 0x58, 0x85, 0x41, 0x62, 0xed, 0x46, 0x50,
-	0x66, 0x02, 0x41, 0xc9, 0xf5, 0x2b, 0x03, 0xd0, 0x1e, 0xf1, 0x5a, 0x4f, 0x49, 0x10, 0xe0, 0xc3,
-	0x58, 0x4b, 0xef, 0xc1, 0xa5, 0x8e, 0x5c, 0x51, 0x82, 0xbd, 0xa9, 0x23, 0x1c, 0x21, 0x45, 0xb0,
-	0xa1, 0xa2, 0x62, 0x0e, 0xc3, 0x14, 0xd5, 0x93, 0xa8, 0x07, 0xaf, 0x04, 0xfa, 0x9d, 0x01, 0xf3,
-	0x29, 0x81, 0x26, 0xa0, 0xa9, 0x1b, 0x00, 0x4a, 0x42, 0xcb, 0x69, 0x09, 0xb9, 0x2a, 0x66, 0x45,
-	0xad, 0x6c, 0xb7, 0xd0, 0x1d, 0x98, 0xe5, 0x0c, 0x7b, 0x01, 0xb6, 0x43, 0x39, 0x42, 0x90, 0x82,
-	0x00, 0xb9, 0x9c, 0x58, 0xdd, 0x6e, 0xed, 0x14, 0xcb, 0xc5, 0x6a, 0xa3, 0xfe, 0x4f, 0x03, 0xae,
-	0x0a, 0x43, 0x36, 0x82, 0xc0, 0x39, 0xf4, 0x3a, 0xc4, 0xe3, 0x17, 0x71, 0xad, 0x35, 0x98, 0x3a,
-	0x64, 0xb4, 0xeb, 0x2b, 0x6d, 0x8d, 0xc0, 0x11, 0xa0, 0xe8, 0x4d, 0xa8, 0xd8, 0xae, 0x43, 0x3c,
-	0xde, 0x13, 0xb5, 0x2c, 0x17, 0xb6, 0x5b, 0x69, 0x5f, 0x2d, 0x9e, 0xd9, 0x57, 0xa7, 0xa4, 0x09,
-	0xae, 0x65, 0x8e, 0x38, 0x01, 0x33, 0xfc, 0x18, 0xa6, 0x71, 0x4c, 0x31, 0xf2, 0xd8, 0x9b, 0x3a,
-	0x02, 0x09, 0xc6, 0x49, 0x14, 0xe5, 0x22, 0xff, 0x28, 0xc2, 0x15, 0x93, 0xd8, 0xc4, 0x39, 0x26,
-	0x7d, 0x6e, 0x1b, 0x6b, 0xd3, 0x38, 0xa7, 0x36, 0xf3, 0x59, 0x6d, 0xf6, 0x1c, 0xba, 0x70, 0x36,
-	0x87, 0x46, 0x2f, 0x60, 0xae, 0xed, 0xb8, 0x9c, 0x30, 0x8b, 0x9c, 0xf8, 0x8c, 0x04, 0x41, 0x48,
-	0x44, 0x9a, 0xe4, 0xb6, 0x8e, 0xc8, 0x63, 0x01, 0xbc, 0x15, 0xc3, 0x9a, 0xd5, 0x76, 0xdf, 0x0a,
-	0x6a, 0xc2, 0xac, 0x4d, 0xbd, 0xa0, 0xdb, 0x21, 0x96, 0x4f, 0x5d, 0xc7, 0x3e, 0x15, 0x91, 0x74,
-	0x76, 0xed, 0x96, 0x8e, 0xde, 0x86, 0x84, 0xdc, 0x15, 0x80, 0xe6, 0x65, 0x3b, 0xf9, 0x89, 0x3e,
-	0x86, 0x05, 0x91, 0x77, 0xb0, 0xeb, 0xbc, 0x16, 0xd9, 0xcf, 0x8a, 0xd3, 0x9c, 0x8a, 0xb3, 0xb5,
-	0x28, 0xce, 0x46, 0x89, 0x70, 0x65, 0x3f, 0x82, 0x30, 0xaf, 0xa5, 0x71, 0xe3, 0x8d, 0xf0, 0xaa,
-	0x1d, 0x60, 0x6e, 0x1f, 0x59, 0x81, 0xf3, 0x9a, 0x2c, 0x5c, 0x5a, 0x32, 0xee, 0x4d, 0x99, 0x15,
-	0xb1, 0xb2, 0xe7, 0xbc, 0x26, 0xa8, 0x09, 0xc8, 0xf1, 0x8e, 0x9d, 0xc0, 0x39, 0x70, 0x89, 0x15,
-	0xe5, 0xdd, 0x85, 0xb2, 0xe0, 0xf7, 0x9d, 0x0c, 0xbf, 0x4d, 0x05, 0x60, 0xce, 0xc5, 0x48, 0xd1,
-	0x12, 0xfa, 0x01, 0x00, 0xfe, 0x02, 0x3b, 0x5c, 0x88, 0xbd, 0x50, 0x19, 0x45, 0xa1, 0x22, 0x80,
-	0x43, 0x39, 0x43, 0x83, 0xb7, 0x9d, 0x36, 0xb5, 0xda, 0x2e, 0x3e, 0x5c, 0x80, 0x25, 0xe3, 0x5e,
-	0xd9, 0x2c, 0x87, 0x0b, 0x8f, 0x5d, 0x7c, 0xb8, 0x53, 0x2c, 0x4f, 0x57, 0x1b, 0xf5, 0x3f, 0xe4,
-	0xe1, 0x6a, 0xbf, 0x87, 0x4d, 0xe0, 0x02, 0x7c, 0x1f, 0xca, 0x2a, 0xea, 0x44, 0xde, 0x3f, 0x34,
-	0xac, 0xc6, 0xc0, 0x68, 0x1b, 0x50, 0x8b, 0xb8, 0xce, 0x31, 0x61, 0xa7, 0x09, 0x33, 0x15, 0x46,
-	0x9a, 0x69, 0x2e, 0xc2, 0xea, 0x19, 0x48, 0x6f, 0x81, 0xe2, 0xd9, 0x2d, 0xa0, 0x82, 0xc5, 0xd7,
-	0x79, 0x98, 0x6b, 0xd8, 0xaf, 0x26, 0x70, 0x11, 0xe3, 0xf0, 0x99, 0x1f, 0x3f, 0x7c, 0x0e, 0x0d,
-	0x85, 0x77, 0x61, 0x96, 0x85, 0x46, 0xf4, 0xb9, 0x75, 0x84, 0xbd, 0x96, 0x4b, 0xc4, 0x31, 0x2b,
-	0xcd, 0x9c, 0x79, 0x59, 0xad, 0x37, 0xc5, 0x32, 0x5a, 0x80, 0x12, 0x6d, 0xb7, 0x03, 0xc2, 0xc5,
-	0x6d, 0x2a, 0x34, 0x73, 0xa6, 0xfa, 0xee, 0xcb, 0x1c, 0xa5, 0xbe, 0xcc, 0xb1, 0x5e, 0x86, 0x92,
-	0xa4, 0xac, 0x4a, 0x8e, 0x97, 0x80, 0x92, 0xba, 0xb8, 0xb8, 0xcb, 0xec, 0x14, 0xcb, 0xf9, 0x6a,
-	0xa3, 0xfe, 0x97, 0x3c, 0xa0, 0x67, 0xf8, 0xff, 0x53, 0xcb, 0x77, 0xf4, 0x5a, 0xee, 0xd7, 0x71,
-	0x5a, 0x93, 0x53, 0xfd, 0x39, 0xf8, 0x6d, 0xa8, 0xc6, 0x1e, 0x8e, 0x39, 0x27, 0x1d, 0x9f, 0x0b,
-	0x75, 0x4f, 0x99, 0x6f, 0x44, 0xeb, 0x0d, 0xb9, 0x8c, 0xd6, 0xe0, 0x4a, 0x07, 0x9f, 0x58, 0xfd,
-	0xe0, 0x81, 0x8a, 0x36, 0xf3, 0x1d, 0x7c, 0xb2, 0x99, 0x46, 0x09, 0x13, 0x47, 0xb9, 0xda, 0xa8,
-	0x7f, 0x02, 0xf3, 0x29, 0x2d, 0x4e, 0xcc, 0x3e, 0xdf, 0xe4, 0x61, 0xf9, 0x31, 0x65, 0x5f, 0x60,
-	0x16, 0xd5, 0x2d, 0xfb, 0x74, 0x93, 0xe0, 0xd6, 0x13, 0xc2, 0x39, 0x61, 0x2f, 0xba, 0xa4, 0xfb,
-	0xad, 0xcd, 0xce, 0x69, 0xb3, 0x57, 0x70, 0x77, 0xa4, 0x66, 0x27, 0x66, 0xc7, 0xff, 0x18, 0x50,
-	0x6d, 0x12, 0xcc, 0xf8, 0x01, 0xc1, 0x71, 0x59, 0x97, 0xd2, 0xa4, 0xd1, 0xa7, 0xc9, 0x8f, 0xe0,
-	0xb2, 0xcf, 0x68, 0xab, 0x6b, 0x13, 0x66, 0xb5, 0x30, 0xc7, 0xca, 0x44, 0x4b, 0xda, 0x22, 0x41,
-	0x01, 0x6e, 0x62, 0x8e, 0x9b, 0x39, 0x73, 0xc6, 0x4f, 0x7c, 0x87, 0x84, 0x54, 0x82, 0x56, 0x84,
-	0x0a, 0x83, 0x09, 0xa9, 0xc4, 0x1e, 0x13, 0xb2, 0x13, 0xdf, 0xe9, 0xf4, 0x56, 0x4c, 0xa7, 0xb7,
-	0xf5, 0xcb, 0x30, 0xad, 0xce, 0x12, 0xf2, 0x50, 0x21, 0xfc, 0x63, 0x98, 0x4b, 0x1c, 0x7a, 0x62,
-	0xca, 0xa4, 0x80, 0x9a, 0x04, 0xbb, 0xfc, 0x68, 0xe3, 0x88, 0xd8, 0xaf, 0x2e, 0xe2, 0xff, 0x8b,
-	0xb1, 0xd4, 0x47, 0x34, 0xe0, 0xaa, 0x48, 0x03, 0xb9, 0xd4, 0xa4, 0x01, 0x57, 0x75, 0xe1, 0x27,
-	0x30, 0x9f, 0x62, 0x38, 0xb1, 0x93, 0xfc, 0xb9, 0x00, 0x57, 0xb6, 0xbc, 0xd6, 0x7e, 0xaf, 0x1d,
-	0xb8, 0xc8, 0x69, 0x26, 0xd2, 0x8d, 0x20, 0x2b, 0xec, 0xc2, 0x03, 0xea, 0x76, 0xe3, 0xfc, 0x3d,
-	0xbb, 0xf6, 0xa3, 0x01, 0x85, 0x7e, 0x56, 0xf0, 0x95, 0xd4, 0x52, 0x44, 0xc6, 0x4c, 0x90, 0x44,
-	0x4d, 0x28, 0x49, 0xb9, 0x55, 0x89, 0xf9, 0xce, 0xf8, 0xc4, 0xf7, 0xe4, 0x79, 0x15, 0x3e, 0x7a,
-	0x0b, 0x42, 0xd9, 0x6d, 0x62, 0xd9, 0xd4, 0xe3, 0xe4, 0x84, 0xab, 0x3c, 0x3a, 0x23, 0x16, 0x37,
-	0xe4, 0x5a, 0xfd, 0x01, 0x5c, 0xd1, 0xca, 0x84, 0x00, 0x4a, 0x1b, 0xcf, 0x9f, 0x3e, 0xdd, 0xde,
-	0xaf, 0xe6, 0xd0, 0x0c, 0x94, 0xcd, 0xe7, 0x4f, 0x9e, 0xac, 0x37, 0x36, 0x7e, 0x52, 0x35, 0xea,
-	0xcb, 0x50, 0x92, 0x9c, 0x04, 0xcc, 0x93, 0xed, 0xad, 0x67, 0x21, 0x4c, 0x15, 0x66, 0xf6, 0xb6,
-	0xcc, 0x97, 0x5b, 0xa6, 0xb5, 0xd1, 0xdc, 0x0a, 0xe1, 0x54, 0x6e, 0xfe, 0x14, 0xae, 0xf6, 0x4b,
-	0x3b, 0x31, 0x07, 0xf9, 0x5b, 0xd4, 0xdd, 0x3f, 0x17, 0x65, 0x43, 0xe4, 0x1d, 0xa9, 0xee, 0xc1,
-	0x38, 0x63, 0xf7, 0xf0, 0x21, 0x94, 0x54, 0x89, 0x9f, 0x17, 0xfa, 0xbf, 0xa3, 0xc3, 0x4c, 0x30,
-	0x55, 0x65, 0xbe, 0x42, 0x42, 0xef, 0x03, 0x84, 0x95, 0xa2, 0x25, 0x3a, 0xbb, 0x31, 0x4a, 0xc5,
-	0x4a, 0x08, 0xbd, 0x1b, 0x02, 0xab, 0x46, 0x97, 0xc2, 0x7c, 0xea, 0x48, 0x13, 0xa8, 0x7f, 0xaf,
-	0xc6, 0x75, 0x56, 0x78, 0xa4, 0x42, 0x54, 0x65, 0xa9, 0xeb, 0xfb, 0xaf, 0x3c, 0xa0, 0xdd, 0xae,
-	0xeb, 0x4e, 0xa0, 0xc8, 0xb9, 0xc8, 0x1c, 0x22, 0x21, 0x65, 0x21, 0x29, 0x65, 0x5f, 0x6b, 0x53,
-	0xec, 0x6f, 0x6d, 0xd2, 0x0d, 0xc9, 0xd4, 0x19, 0x1a, 0x12, 0x6d, 0x9f, 0x58, 0xba, 0x50, 0x9f,
-	0x98, 0xca, 0x59, 0x97, 0xd2, 0x39, 0x4b, 0x25, 0xd6, 0x7f, 0x1b, 0x30, 0x9f, 0x52, 0xf7, 0x84,
-	0x06, 0x2d, 0x8e, 0x67, 0xa5, 0x8c, 0x5c, 0xe9, 0x38, 0x9e, 0xf4, 0xa1, 0x30, 0x8e, 0x7b, 0xe4,
-	0x84, 0x5b, 0x29, 0xf5, 0x42, 0xb8, 0xf4, 0xbc, 0x57, 0x6e, 0xe3, 0x93, 0x68, 0xbf, 0xa8, 0xf0,
-	0xf1, 0x89, 0xda, 0x4e, 0xf6, 0x4f, 0x53, 0x67, 0xe8, 0x9f, 0x76, 0x8a, 0xe5, 0x52, 0xb5, 0x51,
-	0x9f, 0x87, 0xe9, 0x67, 0x94, 0xfa, 0xa1, 0xcc, 0xd8, 0x0b, 0xd5, 0x60, 0x54, 0x1b, 0xf5, 0x0d,
-	0xb8, 0xbe, 0xcb, 0x1c, 0x8f, 0xef, 0x1f, 0x31, 0x82, 0x5b, 0x7b, 0x1c, 0xdb, 0xaf, 0xf6, 0x65,
-	0x5c, 0x12, 0x50, 0xa1, 0x48, 0xb6, 0xfc, 0xd9, 0x4b, 0xff, 0x15, 0xb5, 0x22, 0x74, 0x19, 0xde,
-	0x7f, 0x06, 0x37, 0x4c, 0xe2, 0x53, 0x96, 0xa1, 0x12, 0x39, 0xf1, 0x70, 0x2a, 0xe8, 0x7b, 0x80,
-	0xb8, 0xc0, 0xb4, 0x82, 0x10, 0xd5, 0x12, 0x91, 0x51, 0xa5, 0x86, 0x2a, 0xef, 0xa3, 0xa9, 0xae,
-	0xcb, 0x01, 0xdc, 0x1c, 0xc4, 0x73, 0x62, 0x71, 0xed, 0xe7, 0xb0, 0xf8, 0x92, 0x30, 0xa7, 0x7d,
-	0xaa, 0x54, 0x2a, 0x0b, 0x0f, 0x3f, 0x74, 0xe0, 0xf1, 0xf4, 0x93, 0x1c, 0x24, 0xe6, 0xc7, 0x1f,
-	0x24, 0xaa, 0x23, 0x9e, 0xc0, 0x1d, 0x79, 0xc4, 0x2c, 0x7b, 0x93, 0x04, 0x5d, 0x97, 0x8f, 0xa9,
-	0xde, 0xde, 0xfc, 0x3c, 0x3f, 0x6a, 0x7e, 0xae, 0x38, 0xff, 0x0c, 0x96, 0x47, 0x71, 0x9e, 0x98,
-	0x92, 0xff, 0x64, 0xc0, 0x2d, 0x93, 0xd8, 0xf4, 0x98, 0xb0, 0xe7, 0xcc, 0x3f, 0xc2, 0x1e, 0x49,
-	0x66, 0xa9, 0x48, 0xcf, 0x18, 0x6e, 0x52, 0xb5, 0x6b, 0x25, 0x2a, 0x01, 0xec, 0x5a, 0x67, 0x18,
-	0xd4, 0x5e, 0xa7, 0x59, 0x06, 0x38, 0x8a, 0x00, 0x9a, 0xca, 0x23, 0xaf, 0x9f, 0x83, 0x86, 0x1a,
-	0xfa, 0x2a, 0x8c, 0xd6, 0xd4, 0x75, 0x95, 0x7c, 0x63, 0x15, 0xcb, 0xef, 0x42, 0x49, 0x34, 0x27,
-	0xd1, 0xf4, 0x63, 0x78, 0x2c, 0x57, 0xb0, 0x68, 0x0b, 0x66, 0xe3, 0x12, 0x5b, 0x66, 0x82, 0xc2,
-	0xe8, 0x4c, 0x10, 0x36, 0xf9, 0x11, 0xd6, 0x47, 0x22, 0x27, 0x6c, 0xc5, 0xa3, 0xb3, 0x88, 0x4c,
-	0x71, 0x3c, 0x32, 0x11, 0x96, 0x20, 0xb3, 0x7e, 0x49, 0xa5, 0x23, 0x55, 0x3b, 0xff, 0xb7, 0x00,
-	0xf3, 0x29, 0x35, 0x28, 0xb7, 0xd8, 0x84, 0x19, 0x8f, 0x52, 0xdf, 0x52, 0x4e, 0xa8, 0x8c, 0xb3,
-	0xa8, 0xe3, 0x95, 0x88, 0x49, 0xcd, 0x9c, 0x39, 0xed, 0xf5, 0x3e, 0xd1, 0x6b, 0x58, 0xf2, 0xc3,
-	0xe0, 0x64, 0x65, 0xa3, 0x43, 0x4c, 0x59, 0xba, 0xf4, 0x3b, 0xfa, 0x86, 0x63, 0x70, 0x60, 0x6b,
-	0xe6, 0xcc, 0xeb, 0xfe, 0xb0, 0xc0, 0xf7, 0xa5, 0x01, 0xf5, 0x63, 0x71, 0xf9, 0x23, 0x0f, 0xb3,
-	0xec, 0xde, 0x2d, 0x88, 0xd9, 0x4b, 0x5b, 0x3c, 0xd4, 0xb1, 0x1f, 0x11, 0x3a, 0x9a, 0x39, 0x73,
-	0xf1, 0x78, 0x44, 0x74, 0xf9, 0xa5, 0x01, 0xb7, 0x99, 0xbc, 0x1b, 0x96, 0xce, 0xfd, 0x63, 0x31,
-	0xa4, 0x2d, 0xdf, 0xd3, 0xdb, 0x72, 0xc4, 0xdd, 0x6a, 0xe6, 0xcc, 0x5b, 0x6c, 0x14, 0xd0, 0x7a,
-	0x09, 0x8a, 0xfc, 0xd4, 0x27, 0xca, 0xee, 0x7f, 0x37, 0xe0, 0xe6, 0x33, 0xca, 0x9d, 0xf6, 0xe9,
-	0x86, 0xf0, 0xee, 0x7d, 0xc2, 0x3a, 0x8e, 0x87, 0x93, 0xbd, 0x41, 0xd6, 0x6f, 0x8d, 0xc9, 0xf8,
-	0x6d, 0xfe, 0x1c, 0x7e, 0x3b, 0x74, 0x1e, 0x90, 0x74, 0xea, 0xb0, 0xf4, 0xb3, 0x61, 0x71, 0xe0,
-	0xd9, 0x26, 0x15, 0xf6, 0xd6, 0xbe, 0x9c, 0x85, 0xaa, 0x34, 0xbc, 0xe3, 0x1d, 0xee, 0xc9, 0x57,
-	0x5b, 0xf4, 0x19, 0x40, 0xef, 0x95, 0x0c, 0x0d, 0x2e, 0x79, 0x93, 0x4f, 0x7a, 0xb5, 0xe5, 0x51,
-	0x60, 0x52, 0x8a, 0x7a, 0x0e, 0x7d, 0x0a, 0x95, 0xb8, 0xd3, 0x45, 0xda, 0xda, 0xaa, 0xbf, 0xfb,
-	0xaf, 0xdd, 0x19, 0x01, 0x15, 0xd3, 0xfe, 0x29, 0x4c, 0x27, 0xba, 0x4f, 0xb4, 0x3c, 0x00, 0xaf,
-	0xaf, 0x1f, 0xae, 0xdd, 0x1d, 0x09, 0x97, 0xe4, 0x90, 0x78, 0x19, 0xd3, 0x73, 0xc8, 0xbe, 0xe5,
-	0xe9, 0x39, 0x68, 0x9e, 0xd8, 0xea, 0x39, 0xe4, 0xc2, 0x1b, 0x7d, 0x0f, 0x3f, 0xe8, 0xfe, 0x40,
-	0xe5, 0x66, 0x1e, 0xc0, 0x6a, 0xdf, 0x1d, 0x0b, 0x36, 0xe6, 0xe6, 0xc0, 0x6c, 0x7a, 0xc8, 0x8e,
-	0xde, 0x1e, 0x70, 0x7f, 0xb3, 0x4f, 0x3d, 0xb5, 0xfb, 0xe3, 0x80, 0xc6, 0xac, 0x3e, 0x03, 0xe8,
-	0x0d, 0x66, 0xf5, 0x7e, 0x95, 0x19, 0x62, 0xeb, 0xfd, 0x2a, 0x3b, 0xdf, 0x95, 0x96, 0x49, 0x0c,
-	0x16, 0xf5, 0x96, 0xc9, 0xce, 0x6f, 0xf5, 0x96, 0xd1, 0x4c, 0x28, 0xeb, 0x39, 0xf4, 0x5b, 0x03,
-	0x16, 0x47, 0xcc, 0xc1, 0xd0, 0x23, 0x6d, 0xb3, 0x30, 0xd6, 0x58, 0xb2, 0xf6, 0xc1, 0xb9, 0x70,
-	0x93, 0xa6, 0x4c, 0x37, 0xd7, 0x7a, 0x53, 0x6a, 0xc7, 0x05, 0x7a, 0x53, 0xea, 0x7b, 0x75, 0xa9,
-	0xeb, 0x44, 0x5f, 0x8a, 0x96, 0x47, 0xb4, 0xc5, 0x43, 0x75, 0xad, 0x69, 0x70, 0x25, 0x87, 0x44,
-	0x63, 0xa4, 0xe7, 0x90, 0x6d, 0x54, 0xf5, 0x1c, 0x34, 0x1d, 0x96, 0xe2, 0xd0, 0x2b, 0x1a, 0x06,
-	0x70, 0xc8, 0x14, 0x57, 0x03, 0x38, 0x64, 0xab, 0x8f, 0x7a, 0x0e, 0xfd, 0xc2, 0x80, 0xab, 0xfa,
-	0xf6, 0x00, 0x3d, 0xd0, 0xdf, 0x9c, 0x21, 0xed, 0x4b, 0x6d, 0xed, 0x2c, 0x28, 0xb1, 0x0c, 0xbf,
-	0x31, 0xa2, 0x16, 0x65, 0x50, 0x15, 0x8d, 0xde, 0x1f, 0x4c, 0x78, 0x44, 0xcd, 0x5f, 0x7b, 0x74,
-	0x1e, 0xd4, 0x58, 0xb6, 0xaf, 0x0c, 0xb8, 0x36, 0x20, 0xc7, 0xa1, 0x35, 0x7d, 0x95, 0x36, 0x2c,
-	0xd9, 0xd7, 0x1e, 0x9e, 0x09, 0x27, 0x12, 0x63, 0xfd, 0x87, 0xa0, 0xf9, 0x87, 0xd2, 0x7a, 0xe5,
-	0xe9, 0x0b, 0x95, 0x10, 0x77, 0x8d, 0xdf, 0x1b, 0xc6, 0x37, 0x86, 0xf1, 0xc7, 0xbc, 0x06, 0xee,
-	0xa0, 0x24, 0xc6, 0x0a, 0x0f, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x3a, 0x60, 0x69, 0x08,
-	0x25, 0x00, 0x00,
-}
-
-// Reference imports to suppress errors if they are not otherwise used.
-var _ context.Context
-var _ grpc.ClientConn
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the grpc package it is being compiled against.
-const _ = grpc.SupportPackageIsVersion4
-
-// MessagingServiceClient is the client API for MessagingService service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type MessagingServiceClient interface {
-	// Queries the route entries of the requested topic in the perspective of the
-	// given endpoints. On success, servers should return a collection of
-	// addressable partitions. Note servers may return customized route entries
-	// based on endpoints provided.
-	//
-	// If the requested topic doesn't exist, returns `NOT_FOUND`.
-	// If the specific endpoints is empty, returns `INVALID_ARGUMENT`.
-	QueryRoute(ctx context.Context, in *QueryRouteRequest, opts ...grpc.CallOption) (*QueryRouteResponse, error)
-	// Producer or consumer sends HeartbeatRequest to servers periodically to
-	// keep-alive. Additionally, it also reports client-side configuration,
-	// including topic subscription, load-balancing group name, etc.
-	//
-	// Returns `OK` if success.
-	//
-	// If a client specifies a language that is not yet supported by servers,
-	// returns `INVALID_ARGUMENT`
-	Heartbeat(ctx context.Context, in *HeartbeatRequest, opts ...grpc.CallOption) (*HeartbeatResponse, error)
-	// Checks the health status of message server, returns `OK` if services are
-	// online and serving. Clients may use this RPC to detect availability of
-	// messaging service, and take isolation actions when necessary.
-	HealthCheck(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (*HealthCheckResponse, error)
-	// Delivers messages to brokers.
-	// Clients may further:
-	// 1. Refine a message destination to topic partition which fulfills parts of
-	// FIFO semantic;
-	// 2. Flag a message as transactional, which keeps it invisible to consumers
-	// until it commits;
-	// 3. Time a message, making it invisible to consumers till specified
-	// time-point;
-	// 4. And more...
-	//
-	// Returns message-id or transaction-id with status `OK` on success.
-	//
-	// If the destination topic doesn't exist, returns `NOT_FOUND`.
-	SendMessage(ctx context.Context, in *SendMessageRequest, opts ...grpc.CallOption) (*SendMessageResponse, error)
-	// Queries the assigned partition route info of a topic for current consumer,
-	// the returned assignment result is decided by server-side load balancer.
-	//
-	// If the corresponding topic doesn't exist, returns `NOT_FOUND`.
-	// If the specific endpoints is empty, returns `INVALID_ARGUMENT`.
-	QueryAssignment(ctx context.Context, in *QueryAssignmentRequest, opts ...grpc.CallOption) (*QueryAssignmentResponse, error)
-	// Receives messages from the server in batch manner, returns a batch of
-	// messages if success. The received messages should be ACKed or NACKed after
-	// processing.
-	//
-	// If the pending concurrent receive requests exceed the quota of the given
-	// consumer group, returns `UNAVAILABLE`. If the upstream store server hangs,
-	// return `DEADLINE_EXCEEDED` in a timely manner. If the corresponding topic
-	// or consumer group doesn't exist, returns `NOT_FOUND`. If there is no new
-	// message in the specific topic, returns `OK` with an empty message set.
-	// Please note that client may suffer from false empty responses.
-	ReceiveMessage(ctx context.Context, in *ReceiveMessageRequest, opts ...grpc.CallOption) (*ReceiveMessageResponse, error)
-	// Acknowledges the message associated with the `receipt_handle` or `offset`
-	// in the `AckMessageRequest`, it means the message has been successfully
-	// processed. Returns `OK` if the message server remove the relevant message
-	// successfully.
-	//
-	// If the given receipt_handle is illegal or out of date, returns
-	// `INVALID_ARGUMENT`.
-	AckMessage(ctx context.Context, in *AckMessageRequest, opts ...grpc.CallOption) (*AckMessageResponse, error)
-	// Signals that the message has not been successfully processed. The message
-	// server should resend the message follow the retry policy defined at
-	// server-side.
-	//
-	// If the corresponding topic or consumer group doesn't exist, returns
-	// `NOT_FOUND`.
-	NackMessage(ctx context.Context, in *NackMessageRequest, opts ...grpc.CallOption) (*NackMessageResponse, error)
-	// Forwards one message to dead letter queue if the DeadLetterPolicy is
-	// triggered by this message at client-side, return `OK` if success.
-	ForwardMessageToDeadLetterQueue(ctx context.Context, in *ForwardMessageToDeadLetterQueueRequest, opts ...grpc.CallOption) (*ForwardMessageToDeadLetterQueueResponse, error)
-	// Commits or rollback one transactional message.
-	EndTransaction(ctx context.Context, in *EndTransactionRequest, opts ...grpc.CallOption) (*EndTransactionResponse, error)
-	// Queries the offset of the specific partition, returns the offset with `OK`
-	// if success. The message server should maintain a numerical offset for each
-	// message in a partition.
-	QueryOffset(ctx context.Context, in *QueryOffsetRequest, opts ...grpc.CallOption) (*QueryOffsetResponse, error)
-	// Pulls messages from the specific partition, returns a set of messages with
-	// next pull offset. The pulled messages can't be ACKed or NACKed, while the
-	// client is responsible for manage offsets for consumer, typically update
-	// consume offset to local memory or a third-party storage service.
-	//
-	// If the pending concurrent receive requests exceed the quota of the given
-	// consumer group, returns `UNAVAILABLE`. If the upstream store server hangs,
-	// return `DEADLINE_EXCEEDED` in a timely manner. If the corresponding topic
-	// or consumer group doesn't exist, returns `NOT_FOUND`. If there is no new
-	// message in the specific topic, returns `OK` with an empty message set.
-	// Please note that client may suffer from false empty responses.
-	PullMessage(ctx context.Context, in *PullMessageRequest, opts ...grpc.CallOption) (*PullMessageResponse, error)
-	// Multiplexing RPC(s) for various polling requests, which issue different
-	// commands to client.
-	//
-	// Sometimes client may need to receive and process the command from server.
-	// To prevent the complexity of streaming RPC(s), a unary RPC using
-	// long-polling is another solution.
-	//
-	// To mark the request-response of corresponding command, `command_id` in
-	// message is recorded in the subsequent RPC(s). For example, after receiving
-	// command of printing thread stack trace, client would send
-	// `ReportMessageConsumptionResultRequest` to server, which contain both of
-	// the stack trace and `command_id`.
-	//
-	// At same time, `NoopCommand` is delivered from server when no new command is
-	// needed, it is essential for client to maintain the ping-pong.
-	//
-	PollCommand(ctx context.Context, in *PollCommandRequest, opts ...grpc.CallOption) (*PollCommandResponse, error)
-	// After receiving the corresponding polling command, the thread stack trace
-	// is reported to the server.
-	ReportThreadStackTrace(ctx context.Context, in *ReportThreadStackTraceRequest, opts ...grpc.CallOption) (*ReportThreadStackTraceResponse, error)
-	// After receiving the corresponding polling command, the consumption result
-	// of appointed message is reported to the server.
-	ReportMessageConsumptionResult(ctx context.Context, in *ReportMessageConsumptionResultRequest, opts ...grpc.CallOption) (*ReportMessageConsumptionResultResponse, error)
-	// Notify the server that the client is terminated.
-	NotifyClientTermination(ctx context.Context, in *NotifyClientTerminationRequest, opts ...grpc.CallOption) (*NotifyClientTerminationResponse, error)
-}
-
-type messagingServiceClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewMessagingServiceClient(cc *grpc.ClientConn) MessagingServiceClient {
-	return &messagingServiceClient{cc}
-}
-
-func (c *messagingServiceClient) QueryRoute(ctx context.Context, in *QueryRouteRequest, opts ...grpc.CallOption) (*QueryRouteResponse, error) {
-	out := new(QueryRouteResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/QueryRoute", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) Heartbeat(ctx context.Context, in *HeartbeatRequest, opts ...grpc.CallOption) (*HeartbeatResponse, error) {
-	out := new(HeartbeatResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/Heartbeat", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) HealthCheck(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (*HealthCheckResponse, error) {
-	out := new(HealthCheckResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/HealthCheck", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) SendMessage(ctx context.Context, in *SendMessageRequest, opts ...grpc.CallOption) (*SendMessageResponse, error) {
-	out := new(SendMessageResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/SendMessage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) QueryAssignment(ctx context.Context, in *QueryAssignmentRequest, opts ...grpc.CallOption) (*QueryAssignmentResponse, error) {
-	out := new(QueryAssignmentResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/QueryAssignment", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) ReceiveMessage(ctx context.Context, in *ReceiveMessageRequest, opts ...grpc.CallOption) (*ReceiveMessageResponse, error) {
-	out := new(ReceiveMessageResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/ReceiveMessage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) AckMessage(ctx context.Context, in *AckMessageRequest, opts ...grpc.CallOption) (*AckMessageResponse, error) {
-	out := new(AckMessageResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/AckMessage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) NackMessage(ctx context.Context, in *NackMessageRequest, opts ...grpc.CallOption) (*NackMessageResponse, error) {
-	out := new(NackMessageResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/NackMessage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) ForwardMessageToDeadLetterQueue(ctx context.Context, in *ForwardMessageToDeadLetterQueueRequest, opts ...grpc.CallOption) (*ForwardMessageToDeadLetterQueueResponse, error) {
-	out := new(ForwardMessageToDeadLetterQueueResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/ForwardMessageToDeadLetterQueue", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) EndTransaction(ctx context.Context, in *EndTransactionRequest, opts ...grpc.CallOption) (*EndTransactionResponse, error) {
-	out := new(EndTransactionResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/EndTransaction", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) QueryOffset(ctx context.Context, in *QueryOffsetRequest, opts ...grpc.CallOption) (*QueryOffsetResponse, error) {
-	out := new(QueryOffsetResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/QueryOffset", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) PullMessage(ctx context.Context, in *PullMessageRequest, opts ...grpc.CallOption) (*PullMessageResponse, error) {
-	out := new(PullMessageResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/PullMessage", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) PollCommand(ctx context.Context, in *PollCommandRequest, opts ...grpc.CallOption) (*PollCommandResponse, error) {
-	out := new(PollCommandResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/PollCommand", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) ReportThreadStackTrace(ctx context.Context, in *ReportThreadStackTraceRequest, opts ...grpc.CallOption) (*ReportThreadStackTraceResponse, error) {
-	out := new(ReportThreadStackTraceResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/ReportThreadStackTrace", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) ReportMessageConsumptionResult(ctx context.Context, in *ReportMessageConsumptionResultRequest, opts ...grpc.CallOption) (*ReportMessageConsumptionResultResponse, error) {
-	out := new(ReportMessageConsumptionResultResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/ReportMessageConsumptionResult", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *messagingServiceClient) NotifyClientTermination(ctx context.Context, in *NotifyClientTerminationRequest, opts ...grpc.CallOption) (*NotifyClientTerminationResponse, error) {
-	out := new(NotifyClientTerminationResponse)
-	err := c.cc.Invoke(ctx, "/apache.rocketmq.v1.MessagingService/NotifyClientTermination", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// MessagingServiceServer is the server API for MessagingService service.
-type MessagingServiceServer interface {
-	// Queries the route entries of the requested topic in the perspective of the
-	// given endpoints. On success, servers should return a collection of
-	// addressable partitions. Note servers may return customized route entries
-	// based on endpoints provided.
-	//
-	// If the requested topic doesn't exist, returns `NOT_FOUND`.
-	// If the specific endpoints is empty, returns `INVALID_ARGUMENT`.
-	QueryRoute(context.Context, *QueryRouteRequest) (*QueryRouteResponse, error)
-	// Producer or consumer sends HeartbeatRequest to servers periodically to
-	// keep-alive. Additionally, it also reports client-side configuration,
-	// including topic subscription, load-balancing group name, etc.
-	//
-	// Returns `OK` if success.
-	//
-	// If a client specifies a language that is not yet supported by servers,
-	// returns `INVALID_ARGUMENT`
-	Heartbeat(context.Context, *HeartbeatRequest) (*HeartbeatResponse, error)
-	// Checks the health status of message server, returns `OK` if services are
-	// online and serving. Clients may use this RPC to detect availability of
-	// messaging service, and take isolation actions when necessary.
-	HealthCheck(context.Context, *HealthCheckRequest) (*HealthCheckResponse, error)
-	// Delivers messages to brokers.
-	// Clients may further:
-	// 1. Refine a message destination to topic partition which fulfills parts of
-	// FIFO semantic;
-	// 2. Flag a message as transactional, which keeps it invisible to consumers
-	// until it commits;
-	// 3. Time a message, making it invisible to consumers till specified
-	// time-point;
-	// 4. And more...
-	//
-	// Returns message-id or transaction-id with status `OK` on success.
-	//
-	// If the destination topic doesn't exist, returns `NOT_FOUND`.
-	SendMessage(context.Context, *SendMessageRequest) (*SendMessageResponse, error)
-	// Queries the assigned partition route info of a topic for current consumer,
-	// the returned assignment result is decided by server-side load balancer.
-	//
-	// If the corresponding topic doesn't exist, returns `NOT_FOUND`.
-	// If the specific endpoints is empty, returns `INVALID_ARGUMENT`.
-	QueryAssignment(context.Context, *QueryAssignmentRequest) (*QueryAssignmentResponse, error)
-	// Receives messages from the server in batch manner, returns a batch of
-	// messages if success. The received messages should be ACKed or NACKed after
-	// processing.
-	//
-	// If the pending concurrent receive requests exceed the quota of the given
-	// consumer group, returns `UNAVAILABLE`. If the upstream store server hangs,
-	// return `DEADLINE_EXCEEDED` in a timely manner. If the corresponding topic
-	// or consumer group doesn't exist, returns `NOT_FOUND`. If there is no new
-	// message in the specific topic, returns `OK` with an empty message set.
-	// Please note that client may suffer from false empty responses.
-	ReceiveMessage(context.Context, *ReceiveMessageRequest) (*ReceiveMessageResponse, error)
-	// Acknowledges the message associated with the `receipt_handle` or `offset`
-	// in the `AckMessageRequest`, it means the message has been successfully
-	// processed. Returns `OK` if the message server remove the relevant message
-	// successfully.
-	//
-	// If the given receipt_handle is illegal or out of date, returns
-	// `INVALID_ARGUMENT`.
-	AckMessage(context.Context, *AckMessageRequest) (*AckMessageResponse, error)
-	// Signals that the message has not been successfully processed. The message
-	// server should resend the message follow the retry policy defined at
-	// server-side.
-	//
-	// If the corresponding topic or consumer group doesn't exist, returns
-	// `NOT_FOUND`.
-	NackMessage(context.Context, *NackMessageRequest) (*NackMessageResponse, error)
-	// Forwards one message to dead letter queue if the DeadLetterPolicy is
-	// triggered by this message at client-side, return `OK` if success.
-	ForwardMessageToDeadLetterQueue(context.Context, *ForwardMessageToDeadLetterQueueRequest) (*ForwardMessageToDeadLetterQueueResponse, error)
-	// Commits or rollback one transactional message.
-	EndTransaction(context.Context, *EndTransactionRequest) (*EndTransactionResponse, error)
-	// Queries the offset of the specific partition, returns the offset with `OK`
-	// if success. The message server should maintain a numerical offset for each
-	// message in a partition.
-	QueryOffset(context.Context, *QueryOffsetRequest) (*QueryOffsetResponse, error)
-	// Pulls messages from the specific partition, returns a set of messages with
-	// next pull offset. The pulled messages can't be ACKed or NACKed, while the
-	// client is responsible for manage offsets for consumer, typically update
-	// consume offset to local memory or a third-party storage service.
-	//
-	// If the pending concurrent receive requests exceed the quota of the given
-	// consumer group, returns `UNAVAILABLE`. If the upstream store server hangs,
-	// return `DEADLINE_EXCEEDED` in a timely manner. If the corresponding topic
-	// or consumer group doesn't exist, returns `NOT_FOUND`. If there is no new
-	// message in the specific topic, returns `OK` with an empty message set.
-	// Please note that client may suffer from false empty responses.
-	PullMessage(context.Context, *PullMessageRequest) (*PullMessageResponse, error)
-	// Multiplexing RPC(s) for various polling requests, which issue different
-	// commands to client.
-	//
-	// Sometimes client may need to receive and process the command from server.
-	// To prevent the complexity of streaming RPC(s), a unary RPC using
-	// long-polling is another solution.
-	//
-	// To mark the request-response of corresponding command, `command_id` in
-	// message is recorded in the subsequent RPC(s). For example, after receiving
-	// command of printing thread stack trace, client would send
-	// `ReportMessageConsumptionResultRequest` to server, which contain both of
-	// the stack trace and `command_id`.
-	//
-	// At same time, `NoopCommand` is delivered from server when no new command is
-	// needed, it is essential for client to maintain the ping-pong.
-	//
-	PollCommand(context.Context, *PollCommandRequest) (*PollCommandResponse, error)
-	// After receiving the corresponding polling command, the thread stack trace
-	// is reported to the server.
-	ReportThreadStackTrace(context.Context, *ReportThreadStackTraceRequest) (*ReportThreadStackTraceResponse, error)
-	// After receiving the corresponding polling command, the consumption result
-	// of appointed message is reported to the server.
-	ReportMessageConsumptionResult(context.Context, *ReportMessageConsumptionResultRequest) (*ReportMessageConsumptionResultResponse, error)
-	// Notify the server that the client is terminated.
-	NotifyClientTermination(context.Context, *NotifyClientTerminationRequest) (*NotifyClientTerminationResponse, error)
-}
-
-// UnimplementedMessagingServiceServer can be embedded to have forward compatible implementations.
-type UnimplementedMessagingServiceServer struct {
-}
-
-func (*UnimplementedMessagingServiceServer) QueryRoute(ctx context.Context, req *QueryRouteRequest) (*QueryRouteResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method QueryRoute not implemented")
-}
-func (*UnimplementedMessagingServiceServer) Heartbeat(ctx context.Context, req *HeartbeatRequest) (*HeartbeatResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method Heartbeat not implemented")
-}
-func (*UnimplementedMessagingServiceServer) HealthCheck(ctx context.Context, req *HealthCheckRequest) (*HealthCheckResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method HealthCheck not implemented")
-}
-func (*UnimplementedMessagingServiceServer) SendMessage(ctx context.Context, req *SendMessageRequest) (*SendMessageResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method SendMessage not implemented")
-}
-func (*UnimplementedMessagingServiceServer) QueryAssignment(ctx context.Context, req *QueryAssignmentRequest) (*QueryAssignmentResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method QueryAssignment not implemented")
-}
-func (*UnimplementedMessagingServiceServer) ReceiveMessage(ctx context.Context, req *ReceiveMessageRequest) (*ReceiveMessageResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method ReceiveMessage not implemented")
-}
-func (*UnimplementedMessagingServiceServer) AckMessage(ctx context.Context, req *AckMessageRequest) (*AckMessageResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method AckMessage not implemented")
-}
-func (*UnimplementedMessagingServiceServer) NackMessage(ctx context.Context, req *NackMessageRequest) (*NackMessageResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method NackMessage not implemented")
-}
-func (*UnimplementedMessagingServiceServer) ForwardMessageToDeadLetterQueue(ctx context.Context, req *ForwardMessageToDeadLetterQueueRequest) (*ForwardMessageToDeadLetterQueueResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method ForwardMessageToDeadLetterQueue not implemented")
-}
-func (*UnimplementedMessagingServiceServer) EndTransaction(ctx context.Context, req *EndTransactionRequest) (*EndTransactionResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method EndTransaction not implemented")
-}
-func (*UnimplementedMessagingServiceServer) QueryOffset(ctx context.Context, req *QueryOffsetRequest) (*QueryOffsetResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method QueryOffset not implemented")
-}
-func (*UnimplementedMessagingServiceServer) PullMessage(ctx context.Context, req *PullMessageRequest) (*PullMessageResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method PullMessage not implemented")
-}
-func (*UnimplementedMessagingServiceServer) PollCommand(ctx context.Context, req *PollCommandRequest) (*PollCommandResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method PollCommand not implemented")
-}
-func (*UnimplementedMessagingServiceServer) ReportThreadStackTrace(ctx context.Context, req *ReportThreadStackTraceRequest) (*ReportThreadStackTraceResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method ReportThreadStackTrace not implemented")
-}
-func (*UnimplementedMessagingServiceServer) ReportMessageConsumptionResult(ctx context.Context, req *ReportMessageConsumptionResultRequest) (*ReportMessageConsumptionResultResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method ReportMessageConsumptionResult not implemented")
-}
-func (*UnimplementedMessagingServiceServer) NotifyClientTermination(ctx context.Context, req *NotifyClientTerminationRequest) (*NotifyClientTerminationResponse, error) {
-	return nil, status1.Errorf(codes.Unimplemented, "method NotifyClientTermination not implemented")
-}
-
-func RegisterMessagingServiceServer(s *grpc.Server, srv MessagingServiceServer) {
-	s.RegisterService(&_MessagingService_serviceDesc, srv)
-}
-
-func _MessagingService_QueryRoute_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(QueryRouteRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).QueryRoute(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/QueryRoute",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).QueryRoute(ctx, req.(*QueryRouteRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_Heartbeat_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(HeartbeatRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).Heartbeat(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/Heartbeat",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).Heartbeat(ctx, req.(*HeartbeatRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_HealthCheck_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(HealthCheckRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).HealthCheck(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/HealthCheck",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).HealthCheck(ctx, req.(*HealthCheckRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_SendMessage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(SendMessageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).SendMessage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/SendMessage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).SendMessage(ctx, req.(*SendMessageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_QueryAssignment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(QueryAssignmentRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).QueryAssignment(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/QueryAssignment",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).QueryAssignment(ctx, req.(*QueryAssignmentRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_ReceiveMessage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ReceiveMessageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).ReceiveMessage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/ReceiveMessage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).ReceiveMessage(ctx, req.(*ReceiveMessageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_AckMessage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(AckMessageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).AckMessage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/AckMessage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).AckMessage(ctx, req.(*AckMessageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_NackMessage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(NackMessageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).NackMessage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/NackMessage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).NackMessage(ctx, req.(*NackMessageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_ForwardMessageToDeadLetterQueue_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ForwardMessageToDeadLetterQueueRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).ForwardMessageToDeadLetterQueue(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/ForwardMessageToDeadLetterQueue",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).ForwardMessageToDeadLetterQueue(ctx, req.(*ForwardMessageToDeadLetterQueueRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_EndTransaction_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(EndTransactionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).EndTransaction(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/EndTransaction",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).EndTransaction(ctx, req.(*EndTransactionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_QueryOffset_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(QueryOffsetRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).QueryOffset(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/QueryOffset",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).QueryOffset(ctx, req.(*QueryOffsetRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_PullMessage_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PullMessageRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).PullMessage(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/PullMessage",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).PullMessage(ctx, req.(*PullMessageRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_PollCommand_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PollCommandRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).PollCommand(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/PollCommand",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).PollCommand(ctx, req.(*PollCommandRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_ReportThreadStackTrace_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ReportThreadStackTraceRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).ReportThreadStackTrace(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/ReportThreadStackTrace",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).ReportThreadStackTrace(ctx, req.(*ReportThreadStackTraceRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_ReportMessageConsumptionResult_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ReportMessageConsumptionResultRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).ReportMessageConsumptionResult(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/ReportMessageConsumptionResult",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).ReportMessageConsumptionResult(ctx, req.(*ReportMessageConsumptionResultRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MessagingService_NotifyClientTermination_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(NotifyClientTerminationRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MessagingServiceServer).NotifyClientTermination(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/apache.rocketmq.v1.MessagingService/NotifyClientTermination",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MessagingServiceServer).NotifyClientTermination(ctx, req.(*NotifyClientTerminationRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _MessagingService_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "apache.rocketmq.v1.MessagingService",
-	HandlerType: (*MessagingServiceServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "QueryRoute",
-			Handler:    _MessagingService_QueryRoute_Handler,
-		},
-		{
-			MethodName: "Heartbeat",
-			Handler:    _MessagingService_Heartbeat_Handler,
-		},
-		{
-			MethodName: "HealthCheck",
-			Handler:    _MessagingService_HealthCheck_Handler,
-		},
-		{
-			MethodName: "SendMessage",
-			Handler:    _MessagingService_SendMessage_Handler,
-		},
-		{
-			MethodName: "QueryAssignment",
-			Handler:    _MessagingService_QueryAssignment_Handler,
-		},
-		{
-			MethodName: "ReceiveMessage",
-			Handler:    _MessagingService_ReceiveMessage_Handler,
-		},
-		{
-			MethodName: "AckMessage",
-			Handler:    _MessagingService_AckMessage_Handler,
-		},
-		{
-			MethodName: "NackMessage",
-			Handler:    _MessagingService_NackMessage_Handler,
-		},
-		{
-			MethodName: "ForwardMessageToDeadLetterQueue",
-			Handler:    _MessagingService_ForwardMessageToDeadLetterQueue_Handler,
-		},
-		{
-			MethodName: "EndTransaction",
-			Handler:    _MessagingService_EndTransaction_Handler,
-		},
-		{
-			MethodName: "QueryOffset",
-			Handler:    _MessagingService_QueryOffset_Handler,
-		},
-		{
-			MethodName: "PullMessage",
-			Handler:    _MessagingService_PullMessage_Handler,
-		},
-		{
-			MethodName: "PollCommand",
-			Handler:    _MessagingService_PollCommand_Handler,
-		},
-		{
-			MethodName: "ReportThreadStackTrace",
-			Handler:    _MessagingService_ReportThreadStackTrace_Handler,
-		},
-		{
-			MethodName: "ReportMessageConsumptionResult",
-			Handler:    _MessagingService_ReportMessageConsumptionResult_Handler,
-		},
-		{
-			MethodName: "NotifyClientTermination",
-			Handler:    _MessagingService_NotifyClientTermination_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "apache/rocketmq/v1/service.proto",
-}
diff --git a/golang/protocol/v2/admin.pb.go b/golang/protocol/v2/admin.pb.go
new file mode 100644
index 0000000..8a2b73d
--- /dev/null
+++ b/golang/protocol/v2/admin.pb.go
@@ -0,0 +1,300 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.28.0
+// 	protoc        v3.18.1
+// source: admin.proto
+
+package v2
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+type ChangeLogLevelRequest_Level int32
+
+const (
+	ChangeLogLevelRequest_TRACE ChangeLogLevelRequest_Level = 0
+	ChangeLogLevelRequest_DEBUG ChangeLogLevelRequest_Level = 1
+	ChangeLogLevelRequest_INFO  ChangeLogLevelRequest_Level = 2
+	ChangeLogLevelRequest_WARN  ChangeLogLevelRequest_Level = 3
+	ChangeLogLevelRequest_ERROR ChangeLogLevelRequest_Level = 4
+)
+
+// Enum value maps for ChangeLogLevelRequest_Level.
+var (
+	ChangeLogLevelRequest_Level_name = map[int32]string{
+		0: "TRACE",
+		1: "DEBUG",
+		2: "INFO",
+		3: "WARN",
+		4: "ERROR",
+	}
+	ChangeLogLevelRequest_Level_value = map[string]int32{
+		"TRACE": 0,
+		"DEBUG": 1,
+		"INFO":  2,
+		"WARN":  3,
+		"ERROR": 4,
+	}
+)
+
+func (x ChangeLogLevelRequest_Level) Enum() *ChangeLogLevelRequest_Level {
+	p := new(ChangeLogLevelRequest_Level)
+	*p = x
+	return p
+}
+
+func (x ChangeLogLevelRequest_Level) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (ChangeLogLevelRequest_Level) Descriptor() protoreflect.EnumDescriptor {
+	return file_admin_proto_enumTypes[0].Descriptor()
+}
+
+func (ChangeLogLevelRequest_Level) Type() protoreflect.EnumType {
+	return &file_admin_proto_enumTypes[0]
+}
+
+func (x ChangeLogLevelRequest_Level) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use ChangeLogLevelRequest_Level.Descriptor instead.
+func (ChangeLogLevelRequest_Level) EnumDescriptor() ([]byte, []int) {
+	return file_admin_proto_rawDescGZIP(), []int{0, 0}
+}
+
+type ChangeLogLevelRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Level ChangeLogLevelRequest_Level `protobuf:"varint,1,opt,name=level,proto3,enum=apache.rocketmq.v2.ChangeLogLevelRequest_Level" json:"level,omitempty"`
+}
+
+func (x *ChangeLogLevelRequest) Reset() {
+	*x = ChangeLogLevelRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_admin_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ChangeLogLevelRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ChangeLogLevelRequest) ProtoMessage() {}
+
+func (x *ChangeLogLevelRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_admin_proto_msgTypes[0]
+	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 ChangeLogLevelRequest.ProtoReflect.Descriptor instead.
+func (*ChangeLogLevelRequest) Descriptor() ([]byte, []int) {
+	return file_admin_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *ChangeLogLevelRequest) GetLevel() ChangeLogLevelRequest_Level {
+	if x != nil {
+		return x.Level
+	}
+	return ChangeLogLevelRequest_TRACE
+}
+
+type ChangeLogLevelResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Remark string `protobuf:"bytes,1,opt,name=remark,proto3" json:"remark,omitempty"`
+}
+
+func (x *ChangeLogLevelResponse) Reset() {
+	*x = ChangeLogLevelResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_admin_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ChangeLogLevelResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ChangeLogLevelResponse) ProtoMessage() {}
+
+func (x *ChangeLogLevelResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_admin_proto_msgTypes[1]
+	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 ChangeLogLevelResponse.ProtoReflect.Descriptor instead.
+func (*ChangeLogLevelResponse) Descriptor() ([]byte, []int) {
+	return file_admin_proto_rawDescGZIP(), []int{1}
+}
+
+func (x *ChangeLogLevelResponse) GetRemark() string {
+	if x != nil {
+		return x.Remark
+	}
+	return ""
+}
+
+var File_admin_proto protoreflect.FileDescriptor
+
+var file_admin_proto_rawDesc = []byte{
+	0x0a, 0x0b, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76,
+	0x32, 0x22, 0x9c, 0x01, 0x0a, 0x15, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4c, 0x6f, 0x67, 0x4c,
+	0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x05, 0x6c,
+	0x65, 0x76, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e,
+	0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76,
+	0x65, 0x6c, 0x22, 0x3c, 0x0a, 0x05, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x09, 0x0a, 0x05, 0x54,
+	0x52, 0x41, 0x43, 0x45, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x45, 0x42, 0x55, 0x47, 0x10,
+	0x01, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x4e, 0x46, 0x4f, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x57,
+	0x41, 0x52, 0x4e, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x04,
+	0x22, 0x30, 0x0a, 0x16, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76,
+	0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65,
+	0x6d, 0x61, 0x72, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x61,
+	0x72, 0x6b, 0x32, 0x72, 0x0a, 0x05, 0x41, 0x64, 0x6d, 0x69, 0x6e, 0x12, 0x69, 0x0a, 0x0e, 0x43,
+	0x68, 0x61, 0x6e, 0x67, 0x65, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x29, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e,
+	0x76, 0x32, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65,
+	0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x43, 0x68,
+	0x61, 0x6e, 0x67, 0x65, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70,
+	0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x43, 0x0a, 0x12, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x42, 0x07, 0x4d, 0x51,
+	0x41, 0x64, 0x6d, 0x69, 0x6e, 0x50, 0x01, 0x5a, 0x04, 0x2e, 0x2f, 0x76, 0x32, 0xa0, 0x01, 0x01,
+	0xd8, 0x01, 0x01, 0xf8, 0x01, 0x01, 0xaa, 0x02, 0x12, 0x41, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x52, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x56, 0x32, 0x62, 0x06, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x33,
+}
+
+var (
+	file_admin_proto_rawDescOnce sync.Once
+	file_admin_proto_rawDescData = file_admin_proto_rawDesc
+)
+
+func file_admin_proto_rawDescGZIP() []byte {
+	file_admin_proto_rawDescOnce.Do(func() {
+		file_admin_proto_rawDescData = protoimpl.X.CompressGZIP(file_admin_proto_rawDescData)
+	})
+	return file_admin_proto_rawDescData
+}
+
+var file_admin_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
+var file_admin_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
+var file_admin_proto_goTypes = []interface{}{
+	(ChangeLogLevelRequest_Level)(0), // 0: apache.rocketmq.v2.ChangeLogLevelRequest.Level
+	(*ChangeLogLevelRequest)(nil),    // 1: apache.rocketmq.v2.ChangeLogLevelRequest
+	(*ChangeLogLevelResponse)(nil),   // 2: apache.rocketmq.v2.ChangeLogLevelResponse
+}
+var file_admin_proto_depIdxs = []int32{
+	0, // 0: apache.rocketmq.v2.ChangeLogLevelRequest.level:type_name -> apache.rocketmq.v2.ChangeLogLevelRequest.Level
+	1, // 1: apache.rocketmq.v2.Admin.ChangeLogLevel:input_type -> apache.rocketmq.v2.ChangeLogLevelRequest
+	2, // 2: apache.rocketmq.v2.Admin.ChangeLogLevel:output_type -> apache.rocketmq.v2.ChangeLogLevelResponse
+	2, // [2:3] is the sub-list for method output_type
+	1, // [1:2] is the sub-list for method input_type
+	1, // [1:1] is the sub-list for extension type_name
+	1, // [1:1] is the sub-list for extension extendee
+	0, // [0:1] is the sub-list for field type_name
+}
+
+func init() { file_admin_proto_init() }
+func file_admin_proto_init() {
+	if File_admin_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_admin_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ChangeLogLevelRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_admin_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ChangeLogLevelResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_admin_proto_rawDesc,
+			NumEnums:      1,
+			NumMessages:   2,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_admin_proto_goTypes,
+		DependencyIndexes: file_admin_proto_depIdxs,
+		EnumInfos:         file_admin_proto_enumTypes,
+		MessageInfos:      file_admin_proto_msgTypes,
+	}.Build()
+	File_admin_proto = out.File
+	file_admin_proto_rawDesc = nil
+	file_admin_proto_goTypes = nil
+	file_admin_proto_depIdxs = nil
+}
diff --git a/golang/protocol/v2/admin_grpc.pb.go b/golang/protocol/v2/admin_grpc.pb.go
new file mode 100644
index 0000000..0bcb0c2
--- /dev/null
+++ b/golang/protocol/v2/admin_grpc.pb.go
@@ -0,0 +1,105 @@
+// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
+// versions:
+// - protoc-gen-go-grpc v1.2.0
+// - protoc             v3.18.1
+// source: admin.proto
+
+package v2
+
+import (
+	context "context"
+	grpc "google.golang.org/grpc"
+	codes "google.golang.org/grpc/codes"
+	status "google.golang.org/grpc/status"
+)
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+// Requires gRPC-Go v1.32.0 or later.
+const _ = grpc.SupportPackageIsVersion7
+
+// AdminClient is the client API for Admin service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
+type AdminClient interface {
+	ChangeLogLevel(ctx context.Context, in *ChangeLogLevelRequest, opts ...grpc.CallOption) (*ChangeLogLevelResponse, error)
+}
+
+type adminClient struct {
+	cc grpc.ClientConnInterface
+}
+
+func NewAdminClient(cc grpc.ClientConnInterface) AdminClient {
+	return &adminClient{cc}
+}
+
+func (c *adminClient) ChangeLogLevel(ctx context.Context, in *ChangeLogLevelRequest, opts ...grpc.CallOption) (*ChangeLogLevelResponse, error) {
+	out := new(ChangeLogLevelResponse)
+	err := c.cc.Invoke(ctx, "/apache.rocketmq.v2.Admin/ChangeLogLevel", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// AdminServer is the server API for Admin service.
+// All implementations must embed UnimplementedAdminServer
+// for forward compatibility
+type AdminServer interface {
+	ChangeLogLevel(context.Context, *ChangeLogLevelRequest) (*ChangeLogLevelResponse, error)
+	mustEmbedUnimplementedAdminServer()
+}
+
+// UnimplementedAdminServer must be embedded to have forward compatible implementations.
+type UnimplementedAdminServer struct {
+}
+
+func (UnimplementedAdminServer) ChangeLogLevel(context.Context, *ChangeLogLevelRequest) (*ChangeLogLevelResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method ChangeLogLevel not implemented")
+}
+func (UnimplementedAdminServer) mustEmbedUnimplementedAdminServer() {}
+
+// UnsafeAdminServer may be embedded to opt out of forward compatibility for this service.
+// Use of this interface is not recommended, as added methods to AdminServer will
+// result in compilation errors.
+type UnsafeAdminServer interface {
+	mustEmbedUnimplementedAdminServer()
+}
+
+func RegisterAdminServer(s grpc.ServiceRegistrar, srv AdminServer) {
+	s.RegisterService(&Admin_ServiceDesc, srv)
+}
+
+func _Admin_ChangeLogLevel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ChangeLogLevelRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(AdminServer).ChangeLogLevel(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/apache.rocketmq.v2.Admin/ChangeLogLevel",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(AdminServer).ChangeLogLevel(ctx, req.(*ChangeLogLevelRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+// Admin_ServiceDesc is the grpc.ServiceDesc for Admin service.
+// It's only intended for direct use with grpc.RegisterService,
+// and not to be introspected or modified (even as a copy)
+var Admin_ServiceDesc = grpc.ServiceDesc{
+	ServiceName: "apache.rocketmq.v2.Admin",
+	HandlerType: (*AdminServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "ChangeLogLevel",
+			Handler:    _Admin_ChangeLogLevel_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "admin.proto",
+}
diff --git a/golang/protocol/v2/definition.pb.go b/golang/protocol/v2/definition.pb.go
new file mode 100644
index 0000000..ddca2d0
--- /dev/null
+++ b/golang/protocol/v2/definition.pb.go
@@ -0,0 +1,2635 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.28.0
+// 	protoc        v3.18.1
+// source: definition.proto
+
+package v2
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	durationpb "google.golang.org/protobuf/types/known/durationpb"
+	timestamppb "google.golang.org/protobuf/types/known/timestamppb"
+	reflect "reflect"
+	sync "sync"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+type TransactionResolution int32
+
+const (
+	TransactionResolution_TRANSACTION_RESOLUTION_UNSPECIFIED TransactionResolution = 0
+	TransactionResolution_COMMIT                             TransactionResolution = 1
+	TransactionResolution_ROLLBACK                           TransactionResolution = 2
+)
+
+// Enum value maps for TransactionResolution.
+var (
+	TransactionResolution_name = map[int32]string{
+		0: "TRANSACTION_RESOLUTION_UNSPECIFIED",
+		1: "COMMIT",
+		2: "ROLLBACK",
+	}
+	TransactionResolution_value = map[string]int32{
+		"TRANSACTION_RESOLUTION_UNSPECIFIED": 0,
+		"COMMIT":                             1,
+		"ROLLBACK":                           2,
+	}
+)
+
+func (x TransactionResolution) Enum() *TransactionResolution {
+	p := new(TransactionResolution)
+	*p = x
+	return p
+}
+
+func (x TransactionResolution) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (TransactionResolution) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[0].Descriptor()
+}
+
+func (TransactionResolution) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[0]
+}
+
+func (x TransactionResolution) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use TransactionResolution.Descriptor instead.
+func (TransactionResolution) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{0}
+}
+
+type TransactionSource int32
+
+const (
+	TransactionSource_SOURCE_UNSPECIFIED  TransactionSource = 0
+	TransactionSource_SOURCE_CLIENT       TransactionSource = 1
+	TransactionSource_SOURCE_SERVER_CHECK TransactionSource = 2
+)
+
+// Enum value maps for TransactionSource.
+var (
+	TransactionSource_name = map[int32]string{
+		0: "SOURCE_UNSPECIFIED",
+		1: "SOURCE_CLIENT",
+		2: "SOURCE_SERVER_CHECK",
+	}
+	TransactionSource_value = map[string]int32{
+		"SOURCE_UNSPECIFIED":  0,
+		"SOURCE_CLIENT":       1,
+		"SOURCE_SERVER_CHECK": 2,
+	}
+)
+
+func (x TransactionSource) Enum() *TransactionSource {
+	p := new(TransactionSource)
+	*p = x
+	return p
+}
+
+func (x TransactionSource) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (TransactionSource) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[1].Descriptor()
+}
+
+func (TransactionSource) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[1]
+}
+
+func (x TransactionSource) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use TransactionSource.Descriptor instead.
+func (TransactionSource) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{1}
+}
+
+type Permission int32
+
+const (
+	Permission_PERMISSION_UNSPECIFIED Permission = 0
+	Permission_NONE                   Permission = 1
+	Permission_READ                   Permission = 2
+	Permission_WRITE                  Permission = 3
+	Permission_READ_WRITE             Permission = 4
+)
+
+// Enum value maps for Permission.
+var (
+	Permission_name = map[int32]string{
+		0: "PERMISSION_UNSPECIFIED",
+		1: "NONE",
+		2: "READ",
+		3: "WRITE",
+		4: "READ_WRITE",
+	}
+	Permission_value = map[string]int32{
+		"PERMISSION_UNSPECIFIED": 0,
+		"NONE":                   1,
+		"READ":                   2,
+		"WRITE":                  3,
+		"READ_WRITE":             4,
+	}
+)
+
+func (x Permission) Enum() *Permission {
+	p := new(Permission)
+	*p = x
+	return p
+}
+
+func (x Permission) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (Permission) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[2].Descriptor()
+}
+
+func (Permission) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[2]
+}
+
+func (x Permission) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use Permission.Descriptor instead.
+func (Permission) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{2}
+}
+
+type FilterType int32
+
+const (
+	FilterType_FILTER_TYPE_UNSPECIFIED FilterType = 0
+	FilterType_TAG                     FilterType = 1
+	FilterType_SQL                     FilterType = 2
+)
+
+// Enum value maps for FilterType.
+var (
+	FilterType_name = map[int32]string{
+		0: "FILTER_TYPE_UNSPECIFIED",
+		1: "TAG",
+		2: "SQL",
+	}
+	FilterType_value = map[string]int32{
+		"FILTER_TYPE_UNSPECIFIED": 0,
+		"TAG":                     1,
+		"SQL":                     2,
+	}
+)
+
+func (x FilterType) Enum() *FilterType {
+	p := new(FilterType)
+	*p = x
+	return p
+}
+
+func (x FilterType) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (FilterType) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[3].Descriptor()
+}
+
+func (FilterType) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[3]
+}
+
+func (x FilterType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use FilterType.Descriptor instead.
+func (FilterType) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{3}
+}
+
+type AddressScheme int32
+
+const (
+	AddressScheme_ADDRESS_SCHEME_UNSPECIFIED AddressScheme = 0
+	AddressScheme_IPv4                       AddressScheme = 1
+	AddressScheme_IPv6                       AddressScheme = 2
+	AddressScheme_DOMAIN_NAME                AddressScheme = 3
+)
+
+// Enum value maps for AddressScheme.
+var (
+	AddressScheme_name = map[int32]string{
+		0: "ADDRESS_SCHEME_UNSPECIFIED",
+		1: "IPv4",
+		2: "IPv6",
+		3: "DOMAIN_NAME",
+	}
+	AddressScheme_value = map[string]int32{
+		"ADDRESS_SCHEME_UNSPECIFIED": 0,
+		"IPv4":                       1,
+		"IPv6":                       2,
+		"DOMAIN_NAME":                3,
+	}
+)
+
+func (x AddressScheme) Enum() *AddressScheme {
+	p := new(AddressScheme)
+	*p = x
+	return p
+}
+
+func (x AddressScheme) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (AddressScheme) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[4].Descriptor()
+}
+
+func (AddressScheme) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[4]
+}
+
+func (x AddressScheme) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use AddressScheme.Descriptor instead.
+func (AddressScheme) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{4}
+}
+
+type MessageType int32
+
+const (
+	MessageType_MESSAGE_TYPE_UNSPECIFIED MessageType = 0
+	MessageType_NORMAL                   MessageType = 1
+	// Sequenced message
+	MessageType_FIFO MessageType = 2
+	// Messages that are delivered after the specified duration.
+	MessageType_DELAY MessageType = 3
+	// Messages that are transactional. Only committed messages are delivered to
+	// subscribers.
+	MessageType_TRANSACTION MessageType = 4
+)
+
+// Enum value maps for MessageType.
+var (
+	MessageType_name = map[int32]string{
+		0: "MESSAGE_TYPE_UNSPECIFIED",
+		1: "NORMAL",
+		2: "FIFO",
+		3: "DELAY",
+		4: "TRANSACTION",
+	}
+	MessageType_value = map[string]int32{
+		"MESSAGE_TYPE_UNSPECIFIED": 0,
+		"NORMAL":                   1,
+		"FIFO":                     2,
+		"DELAY":                    3,
+		"TRANSACTION":              4,
+	}
+)
+
+func (x MessageType) Enum() *MessageType {
+	p := new(MessageType)
+	*p = x
+	return p
+}
+
+func (x MessageType) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (MessageType) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[5].Descriptor()
+}
+
+func (MessageType) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[5]
+}
+
+func (x MessageType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use MessageType.Descriptor instead.
+func (MessageType) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{5}
+}
+
+type DigestType int32
+
+const (
+	DigestType_DIGEST_TYPE_UNSPECIFIED DigestType = 0
+	// CRC algorithm achieves goal of detecting random data error with lowest
+	// computation overhead.
+	DigestType_CRC32 DigestType = 1
+	// MD5 algorithm achieves good balance between collision rate and computation
+	// overhead.
+	DigestType_MD5 DigestType = 2
+	// SHA-family has substantially fewer collision with fair amount of
+	// computation.
+	DigestType_SHA1 DigestType = 3
+)
+
+// Enum value maps for DigestType.
+var (
+	DigestType_name = map[int32]string{
+		0: "DIGEST_TYPE_UNSPECIFIED",
+		1: "CRC32",
+		2: "MD5",
+		3: "SHA1",
+	}
+	DigestType_value = map[string]int32{
+		"DIGEST_TYPE_UNSPECIFIED": 0,
+		"CRC32":                   1,
+		"MD5":                     2,
+		"SHA1":                    3,
+	}
+)
+
+func (x DigestType) Enum() *DigestType {
+	p := new(DigestType)
+	*p = x
+	return p
+}
+
+func (x DigestType) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (DigestType) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[6].Descriptor()
+}
+
+func (DigestType) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[6]
+}
+
+func (x DigestType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use DigestType.Descriptor instead.
+func (DigestType) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{6}
+}
+
+type ClientType int32
+
+const (
+	ClientType_CLIENT_TYPE_UNSPECIFIED ClientType = 0
+	ClientType_PRODUCER                ClientType = 1
+	ClientType_PUSH_CONSUMER           ClientType = 2
+	ClientType_SIMPLE_CONSUMER         ClientType = 3
+)
+
+// Enum value maps for ClientType.
+var (
+	ClientType_name = map[int32]string{
+		0: "CLIENT_TYPE_UNSPECIFIED",
+		1: "PRODUCER",
+		2: "PUSH_CONSUMER",
+		3: "SIMPLE_CONSUMER",
+	}
+	ClientType_value = map[string]int32{
+		"CLIENT_TYPE_UNSPECIFIED": 0,
+		"PRODUCER":                1,
+		"PUSH_CONSUMER":           2,
+		"SIMPLE_CONSUMER":         3,
+	}
+)
+
+func (x ClientType) Enum() *ClientType {
+	p := new(ClientType)
+	*p = x
+	return p
+}
+
+func (x ClientType) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (ClientType) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[7].Descriptor()
+}
+
+func (ClientType) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[7]
+}
+
+func (x ClientType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use ClientType.Descriptor instead.
+func (ClientType) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{7}
+}
+
+type Encoding int32
+
+const (
+	Encoding_ENCODING_UNSPECIFIED Encoding = 0
+	Encoding_IDENTITY             Encoding = 1
+	Encoding_GZIP                 Encoding = 2
+)
+
+// Enum value maps for Encoding.
+var (
+	Encoding_name = map[int32]string{
+		0: "ENCODING_UNSPECIFIED",
+		1: "IDENTITY",
+		2: "GZIP",
+	}
+	Encoding_value = map[string]int32{
+		"ENCODING_UNSPECIFIED": 0,
+		"IDENTITY":             1,
+		"GZIP":                 2,
+	}
+)
+
+func (x Encoding) Enum() *Encoding {
+	p := new(Encoding)
+	*p = x
+	return p
+}
+
+func (x Encoding) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (Encoding) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[8].Descriptor()
+}
+
+func (Encoding) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[8]
+}
+
+func (x Encoding) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use Encoding.Descriptor instead.
+func (Encoding) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{8}
+}
+
+type Code int32
+
+const (
+	Code_CODE_UNSPECIFIED Code = 0
+	// Generic code for success.
+	Code_OK Code = 20000
+	// Generic code for multiple return results.
+	Code_MULTIPLE_RESULTS Code = 30000
+	// Generic code for bad request, indicating that required fields or headers are missing.
+	Code_BAD_REQUEST Code = 40000
+	// Format of access point is illegal.
+	Code_ILLEGAL_ACCESS_POINT Code = 40001
+	// Format of topic is illegal.
+	Code_ILLEGAL_TOPIC Code = 40002
+	// Format of consumer group is illegal.
+	Code_ILLEGAL_CONSUMER_GROUP Code = 40003
+	// Format of message tag is illegal.
+	Code_ILLEGAL_MESSAGE_TAG Code = 40004
+	// Format of message key is illegal.
+	Code_ILLEGAL_MESSAGE_KEY Code = 40005
+	// Format of message group is illegal.
+	Code_ILLEGAL_MESSAGE_GROUP Code = 40006
+	// Format of message property key is illegal.
+	Code_ILLEGAL_MESSAGE_PROPERTY_KEY Code = 40007
+	// Transaction id is invalid.
+	Code_INVALID_TRANSACTION_ID Code = 40008
+	// Format of message id is illegal.
+	Code_ILLEGAL_MESSAGE_ID Code = 40009
+	// Format of filter expression is illegal.
+	Code_ILLEGAL_FILTER_EXPRESSION Code = 40010
+	// Receipt handle of message is invalid.
+	Code_INVALID_RECEIPT_HANDLE Code = 40011
+	// Message property conflicts with its type.
+	Code_MESSAGE_PROPERTY_CONFLICT_WITH_TYPE Code = 40012
+	// Client type could not be recognized.
+	Code_UNRECOGNIZED_CLIENT_TYPE Code = 40013
+	// Message is corrupted.
+	Code_MESSAGE_CORRUPTED Code = 40014
+	// Request is rejected due to missing of x-mq-client-id header.
+	Code_CLIENT_ID_REQUIRED Code = 40015
+	// Generic code indicates that the client request lacks valid authentication
+	// credentials for the requested resource.
+	Code_UNAUTHORIZED Code = 40100
+	// Generic code indicates that the account is suspended due to overdue of payment.
+	Code_PAYMENT_REQUIRED Code = 40200
+	// Generic code for the case that user does not have the permission to operate.
+	Code_FORBIDDEN Code = 40300
+	// Generic code for resource not found.
+	Code_NOT_FOUND Code = 40400
+	// Message not found from server.
+	Code_MESSAGE_NOT_FOUND Code = 40401
+	// Topic resource does not exist.
+	Code_TOPIC_NOT_FOUND Code = 40402
+	// Consumer group resource does not exist.
+	Code_CONSUMER_GROUP_NOT_FOUND Code = 40403
+	// Generic code representing client side timeout when connecting to, reading data from, or write data to server.
+	Code_REQUEST_TIMEOUT Code = 40800
+	// Generic code represents that the request entity is larger than limits defined by server.
+	Code_PAYLOAD_TOO_LARGE Code = 41300
+	// Message body size exceeds the threshold.
+	Code_MESSAGE_BODY_TOO_LARGE Code = 41301
+	// Generic code for use cases where pre-conditions are not met.
+	// For example, if a producer instance is used to publish messages without prior start() invocation,
+	// this error code will be raised.
+	Code_PRECONDITION_FAILED Code = 42800
+	// Generic code indicates that too many requests are made in short period of duration.
+	// Requests are throttled.
+	Code_TOO_MANY_REQUESTS Code = 42900
+	// Generic code for the case that the server is unwilling to process the request because its header fields are too large.
+	// The request may be resubmitted after reducing the size of the request header fields.
+	Code_REQUEST_HEADER_FIELDS_TOO_LARGE Code = 43100
+	// Message properties total size exceeds the threshold.
+	Code_MESSAGE_PROPERTIES_TOO_LARGE Code = 43101
+	// Generic code indicates that server/client encountered an unexpected
+	// condition that prevented it from fulfilling the request.
+	Code_INTERNAL_ERROR Code = 50000
+	// Code indicates that the server encountered an unexpected condition
+	// that prevented it from fulfilling the request.
+	// This error response is a generic "catch-all" response.
+	// Usually, this indicates the server cannot find a better alternative
+	// error code to response. Sometimes, server administrators log error
+	// responses like the 500 status code with more details about the request
+	// to prevent the error from happening again in the future.
+	//
+	// See https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/500
+	Code_INTERNAL_SERVER_ERROR Code = 50001
+	// The HA-mechanism is not working now.
+	Code_HA_NOT_AVAILABLE Code = 50002
+	// Generic code means that the server or client does not support the
+	// functionality required to fulfill the request.
+	Code_NOT_IMPLEMENTED Code = 50100
+	// Generic code represents that the server, which acts as a gateway or proxy,
+	// does not get an satisfied response in time from its upstream servers.
+	// See https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/504
+	Code_PROXY_TIMEOUT Code = 50400
+	// Message persistence timeout.
+	Code_MASTER_PERSISTENCE_TIMEOUT Code = 50401
+	// Slave persistence timeout.
+	Code_SLAVE_PERSISTENCE_TIMEOUT Code = 50402
+	// Generic code for unsupported operation.
+	Code_UNSUPPORTED Code = 50500
+	// Operation is not allowed in current version.
+	Code_VERSION_UNSUPPORTED Code = 50501
+	// Not allowed to verify message. Chances are that you are verifying
+	// a FIFO message, as is violating FIFO semantics.
+	Code_VERIFY_FIFO_MESSAGE_UNSUPPORTED Code = 50502
+	// Generic code for failed message consumption.
+	Code_FAILED_TO_CONSUME_MESSAGE Code = 60000
+)
+
+// Enum value maps for Code.
+var (
+	Code_name = map[int32]string{
+		0:     "CODE_UNSPECIFIED",
+		20000: "OK",
+		30000: "MULTIPLE_RESULTS",
+		40000: "BAD_REQUEST",
+		40001: "ILLEGAL_ACCESS_POINT",
+		40002: "ILLEGAL_TOPIC",
+		40003: "ILLEGAL_CONSUMER_GROUP",
+		40004: "ILLEGAL_MESSAGE_TAG",
+		40005: "ILLEGAL_MESSAGE_KEY",
+		40006: "ILLEGAL_MESSAGE_GROUP",
+		40007: "ILLEGAL_MESSAGE_PROPERTY_KEY",
+		40008: "INVALID_TRANSACTION_ID",
+		40009: "ILLEGAL_MESSAGE_ID",
+		40010: "ILLEGAL_FILTER_EXPRESSION",
+		40011: "INVALID_RECEIPT_HANDLE",
+		40012: "MESSAGE_PROPERTY_CONFLICT_WITH_TYPE",
+		40013: "UNRECOGNIZED_CLIENT_TYPE",
+		40014: "MESSAGE_CORRUPTED",
+		40015: "CLIENT_ID_REQUIRED",
+		40100: "UNAUTHORIZED",
+		40200: "PAYMENT_REQUIRED",
+		40300: "FORBIDDEN",
+		40400: "NOT_FOUND",
+		40401: "MESSAGE_NOT_FOUND",
+		40402: "TOPIC_NOT_FOUND",
+		40403: "CONSUMER_GROUP_NOT_FOUND",
+		40800: "REQUEST_TIMEOUT",
+		41300: "PAYLOAD_TOO_LARGE",
+		41301: "MESSAGE_BODY_TOO_LARGE",
+		42800: "PRECONDITION_FAILED",
+		42900: "TOO_MANY_REQUESTS",
+		43100: "REQUEST_HEADER_FIELDS_TOO_LARGE",
+		43101: "MESSAGE_PROPERTIES_TOO_LARGE",
+		50000: "INTERNAL_ERROR",
+		50001: "INTERNAL_SERVER_ERROR",
+		50002: "HA_NOT_AVAILABLE",
+		50100: "NOT_IMPLEMENTED",
+		50400: "PROXY_TIMEOUT",
+		50401: "MASTER_PERSISTENCE_TIMEOUT",
+		50402: "SLAVE_PERSISTENCE_TIMEOUT",
+		50500: "UNSUPPORTED",
+		50501: "VERSION_UNSUPPORTED",
+		50502: "VERIFY_FIFO_MESSAGE_UNSUPPORTED",
+		60000: "FAILED_TO_CONSUME_MESSAGE",
+	}
+	Code_value = map[string]int32{
+		"CODE_UNSPECIFIED":                    0,
+		"OK":                                  20000,
+		"MULTIPLE_RESULTS":                    30000,
+		"BAD_REQUEST":                         40000,
+		"ILLEGAL_ACCESS_POINT":                40001,
+		"ILLEGAL_TOPIC":                       40002,
+		"ILLEGAL_CONSUMER_GROUP":              40003,
+		"ILLEGAL_MESSAGE_TAG":                 40004,
+		"ILLEGAL_MESSAGE_KEY":                 40005,
+		"ILLEGAL_MESSAGE_GROUP":               40006,
+		"ILLEGAL_MESSAGE_PROPERTY_KEY":        40007,
+		"INVALID_TRANSACTION_ID":              40008,
+		"ILLEGAL_MESSAGE_ID":                  40009,
+		"ILLEGAL_FILTER_EXPRESSION":           40010,
+		"INVALID_RECEIPT_HANDLE":              40011,
+		"MESSAGE_PROPERTY_CONFLICT_WITH_TYPE": 40012,
+		"UNRECOGNIZED_CLIENT_TYPE":            40013,
+		"MESSAGE_CORRUPTED":                   40014,
+		"CLIENT_ID_REQUIRED":                  40015,
+		"UNAUTHORIZED":                        40100,
+		"PAYMENT_REQUIRED":                    40200,
+		"FORBIDDEN":                           40300,
+		"NOT_FOUND":                           40400,
+		"MESSAGE_NOT_FOUND":                   40401,
+		"TOPIC_NOT_FOUND":                     40402,
+		"CONSUMER_GROUP_NOT_FOUND":            40403,
+		"REQUEST_TIMEOUT":                     40800,
+		"PAYLOAD_TOO_LARGE":                   41300,
+		"MESSAGE_BODY_TOO_LARGE":              41301,
+		"PRECONDITION_FAILED":                 42800,
+		"TOO_MANY_REQUESTS":                   42900,
+		"REQUEST_HEADER_FIELDS_TOO_LARGE":     43100,
+		"MESSAGE_PROPERTIES_TOO_LARGE":        43101,
+		"INTERNAL_ERROR":                      50000,
+		"INTERNAL_SERVER_ERROR":               50001,
+		"HA_NOT_AVAILABLE":                    50002,
+		"NOT_IMPLEMENTED":                     50100,
+		"PROXY_TIMEOUT":                       50400,
+		"MASTER_PERSISTENCE_TIMEOUT":          50401,
+		"SLAVE_PERSISTENCE_TIMEOUT":           50402,
+		"UNSUPPORTED":                         50500,
+		"VERSION_UNSUPPORTED":                 50501,
+		"VERIFY_FIFO_MESSAGE_UNSUPPORTED":     50502,
+		"FAILED_TO_CONSUME_MESSAGE":           60000,
+	}
+)
+
+func (x Code) Enum() *Code {
+	p := new(Code)
+	*p = x
+	return p
+}
+
+func (x Code) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (Code) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[9].Descriptor()
+}
+
+func (Code) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[9]
+}
+
+func (x Code) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use Code.Descriptor instead.
+func (Code) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{9}
+}
+
+type Language int32
+
+const (
+	Language_LANGUAGE_UNSPECIFIED Language = 0
+	Language_JAVA                 Language = 1
+	Language_CPP                  Language = 2
+	Language_DOT_NET              Language = 3
+	Language_GOLANG               Language = 4
+	Language_RUST                 Language = 5
+)
+
+// Enum value maps for Language.
+var (
+	Language_name = map[int32]string{
+		0: "LANGUAGE_UNSPECIFIED",
+		1: "JAVA",
+		2: "CPP",
+		3: "DOT_NET",
+		4: "GOLANG",
+		5: "RUST",
+	}
+	Language_value = map[string]int32{
+		"LANGUAGE_UNSPECIFIED": 0,
+		"JAVA":                 1,
+		"CPP":                  2,
+		"DOT_NET":              3,
+		"GOLANG":               4,
+		"RUST":                 5,
+	}
+)
+
+func (x Language) Enum() *Language {
+	p := new(Language)
+	*p = x
+	return p
+}
+
+func (x Language) String() string {
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
+}
+
+func (Language) Descriptor() protoreflect.EnumDescriptor {
+	return file_definition_proto_enumTypes[10].Descriptor()
+}
+
+func (Language) Type() protoreflect.EnumType {
+	return &file_definition_proto_enumTypes[10]
+}
+
+func (x Language) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use Language.Descriptor instead.
+func (Language) EnumDescriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{10}
+}
+
+type FilterExpression struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Type       FilterType `protobuf:"varint,1,opt,name=type,proto3,enum=apache.rocketmq.v2.FilterType" json:"type,omitempty"`
+	Expression string     `protobuf:"bytes,2,opt,name=expression,proto3" json:"expression,omitempty"`
+}
+
+func (x *FilterExpression) Reset() {
+	*x = FilterExpression{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FilterExpression) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FilterExpression) ProtoMessage() {}
+
+func (x *FilterExpression) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[0]
+	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 FilterExpression.ProtoReflect.Descriptor instead.
+func (*FilterExpression) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *FilterExpression) GetType() FilterType {
+	if x != nil {
+		return x.Type
+	}
+	return FilterType_FILTER_TYPE_UNSPECIFIED
+}
+
+func (x *FilterExpression) GetExpression() string {
+	if x != nil {
+		return x.Expression
+	}
+	return ""
+}
+
+type RetryPolicy struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	MaxAttempts int32 `protobuf:"varint,1,opt,name=max_attempts,json=maxAttempts,proto3" json:"max_attempts,omitempty"`
+	// Types that are assignable to Strategy:
+	//	*RetryPolicy_ExponentialBackoff
+	//	*RetryPolicy_CustomizedBackoff
+	Strategy isRetryPolicy_Strategy `protobuf_oneof:"strategy"`
+}
+
+func (x *RetryPolicy) Reset() {
+	*x = RetryPolicy{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RetryPolicy) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RetryPolicy) ProtoMessage() {}
+
+func (x *RetryPolicy) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[1]
+	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 RetryPolicy.ProtoReflect.Descriptor instead.
+func (*RetryPolicy) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{1}
+}
+
+func (x *RetryPolicy) GetMaxAttempts() int32 {
+	if x != nil {
+		return x.MaxAttempts
+	}
+	return 0
+}
+
+func (m *RetryPolicy) GetStrategy() isRetryPolicy_Strategy {
+	if m != nil {
+		return m.Strategy
+	}
+	return nil
+}
+
+func (x *RetryPolicy) GetExponentialBackoff() *ExponentialBackoff {
+	if x, ok := x.GetStrategy().(*RetryPolicy_ExponentialBackoff); ok {
+		return x.ExponentialBackoff
+	}
+	return nil
+}
+
+func (x *RetryPolicy) GetCustomizedBackoff() *CustomizedBackoff {
+	if x, ok := x.GetStrategy().(*RetryPolicy_CustomizedBackoff); ok {
+		return x.CustomizedBackoff
+	}
+	return nil
+}
+
+type isRetryPolicy_Strategy interface {
+	isRetryPolicy_Strategy()
+}
+
+type RetryPolicy_ExponentialBackoff struct {
+	ExponentialBackoff *ExponentialBackoff `protobuf:"bytes,2,opt,name=exponential_backoff,json=exponentialBackoff,proto3,oneof"`
+}
+
+type RetryPolicy_CustomizedBackoff struct {
+	CustomizedBackoff *CustomizedBackoff `protobuf:"bytes,3,opt,name=customized_backoff,json=customizedBackoff,proto3,oneof"`
+}
+
+func (*RetryPolicy_ExponentialBackoff) isRetryPolicy_Strategy() {}
+
+func (*RetryPolicy_CustomizedBackoff) isRetryPolicy_Strategy() {}
+
+// https://en.wikipedia.org/wiki/Exponential_backoff
+type ExponentialBackoff struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Initial    *durationpb.Duration `protobuf:"bytes,1,opt,name=initial,proto3" json:"initial,omitempty"`
+	Max        *durationpb.Duration `protobuf:"bytes,2,opt,name=max,proto3" json:"max,omitempty"`
+	Multiplier float32              `protobuf:"fixed32,3,opt,name=multiplier,proto3" json:"multiplier,omitempty"`
+}
+
+func (x *ExponentialBackoff) Reset() {
+	*x = ExponentialBackoff{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExponentialBackoff) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExponentialBackoff) ProtoMessage() {}
+
+func (x *ExponentialBackoff) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[2]
+	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 ExponentialBackoff.ProtoReflect.Descriptor instead.
+func (*ExponentialBackoff) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{2}
+}
+
+func (x *ExponentialBackoff) GetInitial() *durationpb.Duration {
+	if x != nil {
+		return x.Initial
+	}
+	return nil
+}
+
+func (x *ExponentialBackoff) GetMax() *durationpb.Duration {
+	if x != nil {
+		return x.Max
+	}
+	return nil
+}
+
+func (x *ExponentialBackoff) GetMultiplier() float32 {
+	if x != nil {
+		return x.Multiplier
+	}
+	return 0
+}
+
+type CustomizedBackoff struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// To support classic backoff strategy which is arbitary defined by end users.
+	// Typical values are: `1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h`
+	Next []*durationpb.Duration `protobuf:"bytes,1,rep,name=next,proto3" json:"next,omitempty"`
+}
+
+func (x *CustomizedBackoff) Reset() {
+	*x = CustomizedBackoff{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CustomizedBackoff) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CustomizedBackoff) ProtoMessage() {}
+
+func (x *CustomizedBackoff) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[3]
+	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 CustomizedBackoff.ProtoReflect.Descriptor instead.
+func (*CustomizedBackoff) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{3}
+}
+
+func (x *CustomizedBackoff) GetNext() []*durationpb.Duration {
+	if x != nil {
+		return x.Next
+	}
+	return nil
+}
+
+type Resource struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ResourceNamespace string `protobuf:"bytes,1,opt,name=resource_namespace,json=resourceNamespace,proto3" json:"resource_namespace,omitempty"`
+	// Resource name identifier, which remains unique within the abstract resource
+	// namespace.
+	Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
+}
+
+func (x *Resource) Reset() {
+	*x = Resource{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Resource) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Resource) ProtoMessage() {}
+
+func (x *Resource) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[4]
+	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 Resource.ProtoReflect.Descriptor instead.
+func (*Resource) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{4}
+}
+
+func (x *Resource) GetResourceNamespace() string {
+	if x != nil {
+		return x.ResourceNamespace
+	}
+	return ""
+}
+
+func (x *Resource) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+type SubscriptionEntry struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Topic      *Resource         `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+	Expression *FilterExpression `protobuf:"bytes,2,opt,name=expression,proto3" json:"expression,omitempty"`
+}
+
+func (x *SubscriptionEntry) Reset() {
+	*x = SubscriptionEntry{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SubscriptionEntry) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SubscriptionEntry) ProtoMessage() {}
+
+func (x *SubscriptionEntry) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[5]
+	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 SubscriptionEntry.ProtoReflect.Descriptor instead.
+func (*SubscriptionEntry) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{5}
+}
+
+func (x *SubscriptionEntry) GetTopic() *Resource {
+	if x != nil {
+		return x.Topic
+	}
+	return nil
+}
+
+func (x *SubscriptionEntry) GetExpression() *FilterExpression {
+	if x != nil {
+		return x.Expression
+	}
+	return nil
+}
+
+type Address struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Host string `protobuf:"bytes,1,opt,name=host,proto3" json:"host,omitempty"`
+	Port int32  `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
+}
+
+func (x *Address) Reset() {
+	*x = Address{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Address) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Address) ProtoMessage() {}
+
+func (x *Address) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[6]
+	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 Address.ProtoReflect.Descriptor instead.
+func (*Address) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{6}
+}
+
+func (x *Address) GetHost() string {
+	if x != nil {
+		return x.Host
+	}
+	return ""
+}
+
+func (x *Address) GetPort() int32 {
+	if x != nil {
+		return x.Port
+	}
+	return 0
+}
+
+type Endpoints struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Scheme    AddressScheme `protobuf:"varint,1,opt,name=scheme,proto3,enum=apache.rocketmq.v2.AddressScheme" json:"scheme,omitempty"`
+	Addresses []*Address    `protobuf:"bytes,2,rep,name=addresses,proto3" json:"addresses,omitempty"`
+}
+
+func (x *Endpoints) Reset() {
+	*x = Endpoints{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Endpoints) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Endpoints) ProtoMessage() {}
+
+func (x *Endpoints) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Endpoints.ProtoReflect.Descriptor instead.
+func (*Endpoints) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{7}
+}
+
+func (x *Endpoints) GetScheme() AddressScheme {
+	if x != nil {
+		return x.Scheme
+	}
+	return AddressScheme_ADDRESS_SCHEME_UNSPECIFIED
+}
+
+func (x *Endpoints) GetAddresses() []*Address {
+	if x != nil {
+		return x.Addresses
+	}
+	return nil
+}
+
+type Broker struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Name of the broker
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// Broker index. Canonically, index = 0 implies that the broker is playing
+	// leader role while brokers with index > 0 play follower role.
+	Id int32 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
+	// Address of the broker, complying with the following scheme
+	// 1. dns:[//authority/]host[:port]
+	// 2. ipv4:address[:port][,address[:port],...] – IPv4 addresses
+	// 3. ipv6:address[:port][,address[:port],...] – IPv6 addresses
+	Endpoints *Endpoints `protobuf:"bytes,3,opt,name=endpoints,proto3" json:"endpoints,omitempty"`
+}
+
+func (x *Broker) Reset() {
+	*x = Broker{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Broker) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Broker) ProtoMessage() {}
+
+func (x *Broker) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Broker.ProtoReflect.Descriptor instead.
+func (*Broker) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{8}
+}
+
+func (x *Broker) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+func (x *Broker) GetId() int32 {
+	if x != nil {
+		return x.Id
+	}
+	return 0
+}
+
+func (x *Broker) GetEndpoints() *Endpoints {
+	if x != nil {
+		return x.Endpoints
+	}
+	return nil
+}
+
+type MessageQueue struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Topic              *Resource     `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+	Id                 int32         `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
+	Permission         Permission    `protobuf:"varint,3,opt,name=permission,proto3,enum=apache.rocketmq.v2.Permission" json:"permission,omitempty"`
+	Broker             *Broker       `protobuf:"bytes,4,opt,name=broker,proto3" json:"broker,omitempty"`
+	AcceptMessageTypes []MessageType `protobuf:"varint,5,rep,packed,name=accept_message_types,json=acceptMessageTypes,proto3,enum=apache.rocketmq.v2.MessageType" json:"accept_message_types,omitempty"`
+}
+
+func (x *MessageQueue) Reset() {
+	*x = MessageQueue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MessageQueue) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MessageQueue) ProtoMessage() {}
+
+func (x *MessageQueue) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MessageQueue.ProtoReflect.Descriptor instead.
+func (*MessageQueue) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{9}
+}
+
+func (x *MessageQueue) GetTopic() *Resource {
+	if x != nil {
+		return x.Topic
+	}
+	return nil
+}
+
+func (x *MessageQueue) GetId() int32 {
+	if x != nil {
+		return x.Id
+	}
+	return 0
+}
+
+func (x *MessageQueue) GetPermission() Permission {
+	if x != nil {
+		return x.Permission
+	}
+	return Permission_PERMISSION_UNSPECIFIED
+}
+
+func (x *MessageQueue) GetBroker() *Broker {
+	if x != nil {
+		return x.Broker
+	}
+	return nil
+}
+
+func (x *MessageQueue) GetAcceptMessageTypes() []MessageType {
+	if x != nil {
+		return x.AcceptMessageTypes
+	}
+	return nil
+}
+
+// When publishing messages to or subscribing messages from brokers, clients
+// shall include or validate digests of message body to ensure data integrity.
+//
+// For message publishing, when an invalid digest were detected, brokers need
+// respond client with BAD_REQUEST.
+//
+// For messages subscription, when an invalid digest were detected, consumers
+// need to handle this case according to message type:
+// 1) Standard messages should be negatively acknowledged instantly, causing
+// immediate re-delivery; 2) FIFO messages require special RPC, to re-fetch
+// previously acquired messages batch;
+//
+// Message consumption model also affects how invalid digest are handled. When
+// messages are consumed in broadcasting way,
+// TODO: define semantics of invalid-digest-when-broadcasting.
+type Digest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Type     DigestType `protobuf:"varint,1,opt,name=type,proto3,enum=apache.rocketmq.v2.DigestType" json:"type,omitempty"`
+	Checksum string     `protobuf:"bytes,2,opt,name=checksum,proto3" json:"checksum,omitempty"`
+}
+
+func (x *Digest) Reset() {
+	*x = Digest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Digest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Digest) ProtoMessage() {}
+
+func (x *Digest) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Digest.ProtoReflect.Descriptor instead.
+func (*Digest) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{10}
+}
+
+func (x *Digest) GetType() DigestType {
+	if x != nil {
+		return x.Type
+	}
+	return DigestType_DIGEST_TYPE_UNSPECIFIED
+}
+
+func (x *Digest) GetChecksum() string {
+	if x != nil {
+		return x.Checksum
+	}
+	return ""
+}
+
+type SystemProperties struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Tag, which is optional.
+	Tag *string `protobuf:"bytes,1,opt,name=tag,proto3,oneof" json:"tag,omitempty"`
+	// Message keys
+	Keys []string `protobuf:"bytes,2,rep,name=keys,proto3" json:"keys,omitempty"`
+	// Message identifier, client-side generated, remains unique.
+	// if message_id is empty, the send message request will be aborted with
+	// status `INVALID_ARGUMENT`
+	MessageId string `protobuf:"bytes,3,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
+	// Message body digest
+	BodyDigest *Digest `protobuf:"bytes,4,opt,name=body_digest,json=bodyDigest,proto3" json:"body_digest,omitempty"`
+	// Message body encoding. Candidate options are identity, gzip, snappy etc.
+	BodyEncoding Encoding `protobuf:"varint,5,opt,name=body_encoding,json=bodyEncoding,proto3,enum=apache.rocketmq.v2.Encoding" json:"body_encoding,omitempty"`
+	// Message type, normal, FIFO or transactional.
+	MessageType MessageType `protobuf:"varint,6,opt,name=message_type,json=messageType,proto3,enum=apache.rocketmq.v2.MessageType" json:"message_type,omitempty"`
+	// Message born time-point.
+	BornTimestamp *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=born_timestamp,json=bornTimestamp,proto3" json:"born_timestamp,omitempty"`
+	// Message born host. Valid options are IPv4, IPv6 or client host domain name.
+	BornHost string `protobuf:"bytes,8,opt,name=born_host,json=bornHost,proto3" json:"born_host,omitempty"`
+	// Time-point at which the message is stored in the broker, which is absent
+	// for message publishing.
+	StoreTimestamp *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=store_timestamp,json=storeTimestamp,proto3,oneof" json:"store_timestamp,omitempty"`
+	// The broker that stores this message. It may be broker name, IP or arbitrary
+	// identifier that uniquely identify the server.
+	StoreHost string `protobuf:"bytes,10,opt,name=store_host,json=storeHost,proto3" json:"store_host,omitempty"`
+	// Time-point at which broker delivers to clients, which is optional.
+	DeliveryTimestamp *timestamppb.Timestamp `protobuf:"bytes,11,opt,name=delivery_timestamp,json=deliveryTimestamp,proto3,oneof" json:"delivery_timestamp,omitempty"`
+	// If a message is acquired by way of POP, this field holds the receipt,
+	// which is absent for message publishing.
+	// Clients use the receipt to acknowledge or negatively acknowledge the
+	// message.
+	ReceiptHandle *string `protobuf:"bytes,12,opt,name=receipt_handle,json=receiptHandle,proto3,oneof" json:"receipt_handle,omitempty"`
+	// Message queue identifier in which a message is physically stored.
+	QueueId int32 `protobuf:"varint,13,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"`
+	// Message-queue offset at which a message is stored, which is absent for
+	// message publishing.
+	QueueOffset *int64 `protobuf:"varint,14,opt,name=queue_offset,json=queueOffset,proto3,oneof" json:"queue_offset,omitempty"`
+	// Period of time servers would remain invisible once a message is acquired.
+	InvisibleDuration *durationpb.Duration `protobuf:"bytes,15,opt,name=invisible_duration,json=invisibleDuration,proto3,oneof" json:"invisible_duration,omitempty"`
+	// Business code may failed to process messages for the moment. Hence, clients
+	// may request servers to deliver them again using certain back-off strategy,
+	// the attempt is 1 not 0 if message is delivered first time, and it is absent
+	// for message publishing.
+	DeliveryAttempt *int32 `protobuf:"varint,16,opt,name=delivery_attempt,json=deliveryAttempt,proto3,oneof" json:"delivery_attempt,omitempty"`
+	// Define the group name of message in the same topic, which is optional.
+	MessageGroup *string `protobuf:"bytes,17,opt,name=message_group,json=messageGroup,proto3,oneof" json:"message_group,omitempty"`
+	// Trace context for each message, which is optional.
+	TraceContext *string `protobuf:"bytes,18,opt,name=trace_context,json=traceContext,proto3,oneof" json:"trace_context,omitempty"`
+	// If a transactional message stay unresolved for more than
+	// `transaction_orphan_threshold`, it would be regarded as an
+	// orphan. Servers that manages orphan messages would pick up
+	// a capable publisher to resolve
+	OrphanedTransactionRecoveryDuration *durationpb.Duration `protobuf:"bytes,19,opt,name=orphaned_transaction_recovery_duration,json=orphanedTransactionRecoveryDuration,proto3,oneof" json:"orphaned_transaction_recovery_duration,omitempty"`
+}
+
+func (x *SystemProperties) Reset() {
+	*x = SystemProperties{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SystemProperties) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SystemProperties) ProtoMessage() {}
+
+func (x *SystemProperties) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SystemProperties.ProtoReflect.Descriptor instead.
+func (*SystemProperties) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{11}
+}
+
+func (x *SystemProperties) GetTag() string {
+	if x != nil && x.Tag != nil {
+		return *x.Tag
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetKeys() []string {
+	if x != nil {
+		return x.Keys
+	}
+	return nil
+}
+
+func (x *SystemProperties) GetMessageId() string {
+	if x != nil {
+		return x.MessageId
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetBodyDigest() *Digest {
+	if x != nil {
+		return x.BodyDigest
+	}
+	return nil
+}
+
+func (x *SystemProperties) GetBodyEncoding() Encoding {
+	if x != nil {
+		return x.BodyEncoding
+	}
+	return Encoding_ENCODING_UNSPECIFIED
+}
+
+func (x *SystemProperties) GetMessageType() MessageType {
+	if x != nil {
+		return x.MessageType
+	}
+	return MessageType_MESSAGE_TYPE_UNSPECIFIED
+}
+
+func (x *SystemProperties) GetBornTimestamp() *timestamppb.Timestamp {
+	if x != nil {
+		return x.BornTimestamp
+	}
+	return nil
+}
+
+func (x *SystemProperties) GetBornHost() string {
+	if x != nil {
+		return x.BornHost
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetStoreTimestamp() *timestamppb.Timestamp {
+	if x != nil {
+		return x.StoreTimestamp
+	}
+	return nil
+}
+
+func (x *SystemProperties) GetStoreHost() string {
+	if x != nil {
+		return x.StoreHost
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetDeliveryTimestamp() *timestamppb.Timestamp {
+	if x != nil {
+		return x.DeliveryTimestamp
+	}
+	return nil
+}
+
+func (x *SystemProperties) GetReceiptHandle() string {
+	if x != nil && x.ReceiptHandle != nil {
+		return *x.ReceiptHandle
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetQueueId() int32 {
+	if x != nil {
+		return x.QueueId
+	}
+	return 0
+}
+
+func (x *SystemProperties) GetQueueOffset() int64 {
+	if x != nil && x.QueueOffset != nil {
+		return *x.QueueOffset
+	}
+	return 0
+}
+
+func (x *SystemProperties) GetInvisibleDuration() *durationpb.Duration {
+	if x != nil {
+		return x.InvisibleDuration
+	}
+	return nil
+}
+
+func (x *SystemProperties) GetDeliveryAttempt() int32 {
+	if x != nil && x.DeliveryAttempt != nil {
+		return *x.DeliveryAttempt
+	}
+	return 0
+}
+
+func (x *SystemProperties) GetMessageGroup() string {
+	if x != nil && x.MessageGroup != nil {
+		return *x.MessageGroup
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetTraceContext() string {
+	if x != nil && x.TraceContext != nil {
+		return *x.TraceContext
+	}
+	return ""
+}
+
+func (x *SystemProperties) GetOrphanedTransactionRecoveryDuration() *durationpb.Duration {
+	if x != nil {
+		return x.OrphanedTransactionRecoveryDuration
+	}
+	return nil
+}
+
+type Message struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Topic *Resource `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+	// User defined key-value pairs.
+	// If user_properties contain the reserved keys by RocketMQ,
+	// the send message request will be aborted with status `INVALID_ARGUMENT`.
+	// See below links for the reserved keys
+	// https://github.com/apache/rocketmq/blob/master/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java#L58
+	UserProperties   map[string]string `protobuf:"bytes,2,rep,name=user_properties,json=userProperties,proto3" json:"user_properties,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+	SystemProperties *SystemProperties `protobuf:"bytes,3,opt,name=system_properties,json=systemProperties,proto3" json:"system_properties,omitempty"`
+	Body             []byte            `protobuf:"bytes,4,opt,name=body,proto3" json:"body,omitempty"`
+}
+
+func (x *Message) Reset() {
+	*x = Message{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Message) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Message) ProtoMessage() {}
+
+func (x *Message) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Message.ProtoReflect.Descriptor instead.
+func (*Message) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{12}
+}
+
+func (x *Message) GetTopic() *Resource {
+	if x != nil {
+		return x.Topic
+	}
+	return nil
+}
+
+func (x *Message) GetUserProperties() map[string]string {
+	if x != nil {
+		return x.UserProperties
+	}
+	return nil
+}
+
+func (x *Message) GetSystemProperties() *SystemProperties {
+	if x != nil {
+		return x.SystemProperties
+	}
+	return nil
+}
+
+func (x *Message) GetBody() []byte {
+	if x != nil {
+		return x.Body
+	}
+	return nil
+}
+
+type Assignment struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	MessageQueue *MessageQueue `protobuf:"bytes,1,opt,name=message_queue,json=messageQueue,proto3" json:"message_queue,omitempty"`
+}
+
+func (x *Assignment) Reset() {
+	*x = Assignment{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Assignment) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Assignment) ProtoMessage() {}
+
+func (x *Assignment) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Assignment.ProtoReflect.Descriptor instead.
+func (*Assignment) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{13}
+}
+
+func (x *Assignment) GetMessageQueue() *MessageQueue {
+	if x != nil {
+		return x.MessageQueue
+	}
+	return nil
+}
+
+type Status struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Code    Code   `protobuf:"varint,1,opt,name=code,proto3,enum=apache.rocketmq.v2.Code" json:"code,omitempty"`
+	Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"`
+}
+
+func (x *Status) Reset() {
+	*x = Status{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Status) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Status) ProtoMessage() {}
+
+func (x *Status) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Status.ProtoReflect.Descriptor instead.
+func (*Status) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{14}
+}
+
+func (x *Status) GetCode() Code {
+	if x != nil {
+		return x.Code
+	}
+	return Code_CODE_UNSPECIFIED
+}
+
+func (x *Status) GetMessage() string {
+	if x != nil {
+		return x.Message
+	}
+	return ""
+}
+
+// User Agent
+type UA struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// SDK language
+	Language Language `protobuf:"varint,1,opt,name=language,proto3,enum=apache.rocketmq.v2.Language" json:"language,omitempty"`
+	// SDK version
+	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
+	// Platform details, including OS name, version, arch etc.
+	Platform string `protobuf:"bytes,3,opt,name=platform,proto3" json:"platform,omitempty"`
+	// Hostname of the node
+	Hostname string `protobuf:"bytes,4,opt,name=hostname,proto3" json:"hostname,omitempty"`
+}
+
+func (x *UA) Reset() {
+	*x = UA{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_definition_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *UA) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*UA) ProtoMessage() {}
+
+func (x *UA) ProtoReflect() protoreflect.Message {
+	mi := &file_definition_proto_msgTypes[15]
+	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 UA.ProtoReflect.Descriptor instead.
+func (*UA) Descriptor() ([]byte, []int) {
+	return file_definition_proto_rawDescGZIP(), []int{15}
+}
+
+func (x *UA) GetLanguage() Language {
+	if x != nil {
+		return x.Language
+	}
+	return Language_LANGUAGE_UNSPECIFIED
+}
+
+func (x *UA) GetVersion() string {
+	if x != nil {
+		return x.Version
+	}
+	return ""
+}
+
+func (x *UA) GetPlatform() string {
+	if x != nil {
+		return x.Platform
+	}
+	return ""
+}
+
+func (x *UA) GetHostname() string {
+	if x != nil {
+		return x.Hostname
+	}
+	return ""
+}
+
+var File_definition_proto protoreflect.FileDescriptor
+
+var file_definition_proto_rawDesc = []byte{
+	0x0a, 0x10, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x12, 0x12, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65,
+	0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d,
+	0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x66, 0x0a, 0x10, 0x46, 0x69, 0x6c, 0x74, 0x65,
+	0x72, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x0a, 0x04, 0x74,
+	0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x46,
+	0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12,
+	0x1e, 0x0a, 0x0a, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22,
+	0xef, 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x74, 0x72, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12,
+	0x21, 0x0a, 0x0c, 0x6d, 0x61, 0x78, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x6d, 0x61, 0x78, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70,
+	0x74, 0x73, 0x12, 0x59, 0x0a, 0x13, 0x65, 0x78, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x69, 0x61,
+	0x6c, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x26, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d,
+	0x71, 0x2e, 0x76, 0x32, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c,
+	0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x48, 0x00, 0x52, 0x12, 0x65, 0x78, 0x70, 0x6f, 0x6e,
+	0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x12, 0x56, 0x0a,
+	0x12, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x62, 0x61, 0x63, 0x6b,
+	0x6f, 0x66, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x43,
+	0x75, 0x73, 0x74, 0x6f, 0x6d, 0x69, 0x7a, 0x65, 0x64, 0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66,
+	0x48, 0x00, 0x52, 0x11, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x69, 0x7a, 0x65, 0x64, 0x42, 0x61,
+	0x63, 0x6b, 0x6f, 0x66, 0x66, 0x42, 0x0a, 0x0a, 0x08, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67,
+	0x79, 0x22, 0x96, 0x01, 0x0a, 0x12, 0x45, 0x78, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x69, 0x61,
+	0x6c, 0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x12, 0x33, 0x0a, 0x07, 0x69, 0x6e, 0x69, 0x74,
+	0x69, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67,
+	0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61,
+	0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x2b, 0x0a,
+	0x03, 0x6d, 0x61, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x6d, 0x61, 0x78, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x75,
+	0x6c, 0x74, 0x69, 0x70, 0x6c, 0x69, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0a,
+	0x6d, 0x75, 0x6c, 0x74, 0x69, 0x70, 0x6c, 0x69, 0x65, 0x72, 0x22, 0x42, 0x0a, 0x11, 0x43, 0x75,
+	0x73, 0x74, 0x6f, 0x6d, 0x69, 0x7a, 0x65, 0x64, 0x42, 0x61, 0x63, 0x6b, 0x6f, 0x66, 0x66, 0x12,
+	0x2d, 0x0a, 0x04, 0x6e, 0x65, 0x78, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
+	0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x6e, 0x65, 0x78, 0x74, 0x22, 0x4d,
+	0x0a, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x72, 0x65,
+	0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+	0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d,
+	0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x8d, 0x01,
+	0x0a, 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e,
+	0x74, 0x72, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b,
+	0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+	0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x44, 0x0a, 0x0a, 0x65, 0x78, 0x70, 0x72, 0x65,
+	0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32,
+	0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,
+	0x6e, 0x52, 0x0a, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x31, 0x0a,
+	0x07, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04,
+	0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74,
+	0x22, 0x81, 0x01, 0x0a, 0x09, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x39,
+	0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71,
+	0x2e, 0x76, 0x32, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x53, 0x63, 0x68, 0x65, 0x6d,
+	0x65, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x65, 0x12, 0x39, 0x0a, 0x09, 0x61, 0x64, 0x64,
+	0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76,
+	0x32, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 0x09, 0x61, 0x64, 0x64, 0x72, 0x65,
+	0x73, 0x73, 0x65, 0x73, 0x22, 0x69, 0x0a, 0x06, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x12,
+	0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61,
+	0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02,
+	0x69, 0x64, 0x12, 0x3b, 0x0a, 0x09, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18,
+	0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72,
+	0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x45, 0x6e, 0x64, 0x70, 0x6f,
+	0x69, 0x6e, 0x74, 0x73, 0x52, 0x09, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x22,
+	0x99, 0x02, 0x0a, 0x0c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x51, 0x75, 0x65, 0x75, 0x65,
+	0x12, 0x32, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x1c, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d,
+	0x71, 0x2e, 0x76, 0x32, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x05, 0x74,
+	0x6f, 0x70, 0x69, 0x63, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05,
+	0x52, 0x02, 0x69, 0x64, 0x12, 0x3e, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69,
+	0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x50, 0x65,
+	0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73,
+	0x73, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x04,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f,
+	0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72,
+	0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x51, 0x0a, 0x14, 0x61, 0x63, 0x63, 0x65,
+	0x70, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73,
+	0x18, 0x05, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x4d, 0x65, 0x73, 0x73,
+	0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x12, 0x61, 0x63, 0x63, 0x65, 0x70, 0x74, 0x4d,
+	0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x73, 0x22, 0x58, 0x0a, 0x06, 0x44,
+	0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x32, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63,
+	0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x54,
+	0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65,
+	0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x65,
+	0x63, 0x6b, 0x73, 0x75, 0x6d, 0x22, 0xc2, 0x09, 0x0a, 0x10, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d,
+	0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x15, 0x0a, 0x03, 0x74, 0x61,
+	0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x74, 0x61, 0x67, 0x88, 0x01,
+	0x01, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52,
+	0x04, 0x6b, 0x65, 0x79, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+	0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61,
+	0x67, 0x65, 0x49, 0x64, 0x12, 0x3b, 0x0a, 0x0b, 0x62, 0x6f, 0x64, 0x79, 0x5f, 0x64, 0x69, 0x67,
+	0x65, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x44,
+	0x69, 0x67, 0x65, 0x73, 0x74, 0x52, 0x0a, 0x62, 0x6f, 0x64, 0x79, 0x44, 0x69, 0x67, 0x65, 0x73,
+	0x74, 0x12, 0x41, 0x0a, 0x0d, 0x62, 0x6f, 0x64, 0x79, 0x5f, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69,
+	0x6e, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x45, 0x6e,
+	0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x0c, 0x62, 0x6f, 0x64, 0x79, 0x45, 0x6e, 0x63, 0x6f,
+	0x64, 0x69, 0x6e, 0x67, 0x12, 0x42, 0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f,
+	0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e,
+	0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x6d, 0x65, 0x73,
+	0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x41, 0x0a, 0x0e, 0x62, 0x6f, 0x72, 0x6e,
+	0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0d, 0x62, 0x6f,
+	0x72, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x1b, 0x0a, 0x09, 0x62,
+	0x6f, 0x72, 0x6e, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
+	0x62, 0x6f, 0x72, 0x6e, 0x48, 0x6f, 0x73, 0x74, 0x12, 0x48, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72,
+	0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x48, 0x01, 0x52,
+	0x0e, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x88,
+	0x01, 0x01, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x68, 0x6f, 0x73, 0x74,
+	0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x48, 0x6f, 0x73,
+	0x74, 0x12, 0x4e, 0x0a, 0x12, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x79, 0x5f, 0x74, 0x69,
+	0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
+	0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x48, 0x02, 0x52, 0x11, 0x64, 0x65, 0x6c,
+	0x69, 0x76, 0x65, 0x72, 0x79, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x88, 0x01,
+	0x01, 0x12, 0x2a, 0x0a, 0x0e, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x5f, 0x68, 0x61, 0x6e,
+	0x64, 0x6c, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x0d, 0x72, 0x65, 0x63,
+	0x65, 0x69, 0x70, 0x74, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x88, 0x01, 0x01, 0x12, 0x19, 0x0a,
+	0x08, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x05, 0x52,
+	0x07, 0x71, 0x75, 0x65, 0x75, 0x65, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x0c, 0x71, 0x75, 0x65, 0x75,
+	0x65, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x48, 0x04,
+	0x52, 0x0b, 0x71, 0x75, 0x65, 0x75, 0x65, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x88, 0x01, 0x01,
+	0x12, 0x4d, 0x0a, 0x12, 0x69, 0x6e, 0x76, 0x69, 0x73, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x75,
+	0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44,
+	0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x05, 0x52, 0x11, 0x69, 0x6e, 0x76, 0x69, 0x73,
+	0x69, 0x62, 0x6c, 0x65, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12,
+	0x2e, 0x0a, 0x10, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x79, 0x5f, 0x61, 0x74, 0x74, 0x65,
+	0x6d, 0x70, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x05, 0x48, 0x06, 0x52, 0x0f, 0x64, 0x65, 0x6c,
+	0x69, 0x76, 0x65, 0x72, 0x79, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x88, 0x01, 0x01, 0x12,
+	0x28, 0x0a, 0x0d, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70,
+	0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x48, 0x07, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+	0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x88, 0x01, 0x01, 0x12, 0x28, 0x0a, 0x0d, 0x74, 0x72, 0x61,
+	0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x09,
+	0x48, 0x08, 0x52, 0x0c, 0x74, 0x72, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74,
+	0x88, 0x01, 0x01, 0x12, 0x73, 0x0a, 0x26, 0x6f, 0x72, 0x70, 0x68, 0x61, 0x6e, 0x65, 0x64, 0x5f,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x63, 0x6f,
+	0x76, 0x65, 0x72, 0x79, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x13, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x09,
+	0x52, 0x23, 0x6f, 0x72, 0x70, 0x68, 0x61, 0x6e, 0x65, 0x64, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61,
+	0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x44, 0x75, 0x72,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x06, 0x0a, 0x04, 0x5f, 0x74, 0x61, 0x67,
+	0x42, 0x12, 0x0a, 0x10, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73,
+	0x74, 0x61, 0x6d, 0x70, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72,
+	0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x42, 0x11, 0x0a, 0x0f, 0x5f,
+	0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x42, 0x0f,
+	0x0a, 0x0d, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x42,
+	0x15, 0x0a, 0x13, 0x5f, 0x69, 0x6e, 0x76, 0x69, 0x73, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x75,
+	0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x13, 0x0a, 0x11, 0x5f, 0x64, 0x65, 0x6c, 0x69, 0x76,
+	0x65, 0x72, 0x79, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x42, 0x10, 0x0a, 0x0e, 0x5f,
+	0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x10, 0x0a,
+	0x0e, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x42,
+	0x29, 0x0a, 0x27, 0x5f, 0x6f, 0x72, 0x70, 0x68, 0x61, 0x6e, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72,
+	0x79, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xc1, 0x02, 0x0a, 0x07, 0x4d,
+	0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x32, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72,
+	0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75,
+	0x72, 0x63, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x58, 0x0a, 0x0f, 0x75, 0x73,
+	0x65, 0x72, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20,
+	0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63,
+	0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+	0x2e, 0x55, 0x73, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45,
+	0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72,
+	0x74, 0x69, 0x65, 0x73, 0x12, 0x51, 0x0a, 0x11, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x70,
+	0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x24, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d,
+	0x71, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x50, 0x72, 0x6f, 0x70, 0x65,
+	0x72, 0x74, 0x69, 0x65, 0x73, 0x52, 0x10, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x50, 0x72, 0x6f,
+	0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x18,
+	0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x62, 0x6f, 0x64, 0x79, 0x1a, 0x41, 0x0a, 0x13, 0x55,
+	0x73, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74,
+	0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x53,
+	0x0a, 0x0a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x45, 0x0a, 0x0d,
+	0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x71, 0x75, 0x65, 0x75, 0x65, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63,
+	0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
+	0x51, 0x75, 0x65, 0x75, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x51, 0x75,
+	0x65, 0x75, 0x65, 0x22, 0x50, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2c, 0x0a,
+	0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71, 0x2e, 0x76, 0x32,
+	0x2e, 0x43, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d,
+	0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65,
+	0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x90, 0x01, 0x0a, 0x02, 0x55, 0x41, 0x12, 0x38, 0x0a, 0x08,
+	0x6c, 0x61, 0x6e, 0x67, 0x75, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x72, 0x6f, 0x63, 0x6b, 0x65, 0x74, 0x6d, 0x71,
+	0x2e, 0x76, 0x32, 0x2e, 0x4c, 0x61, 0x6e, 0x67, 0x75, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6c, 0x61,
+	0x6e, 0x67, 0x75, 0x61, 0x67, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f,
+	0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e,
+	0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x18, 0x03, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x08, 0x70, 0x6c, 0x61, 0x74, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x1a, 0x0a, 0x08,
+	0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
+	0x68, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x2a, 0x59, 0x0a, 0x15, 0x54, 0x72, 0x61, 0x6e,
+	0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x12, 0x26, 0x0a, 0x22, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e,
+	0x5f, 0x52, 0x45, 0x53, 0x4f, 0x4c, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50,
+	0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d,
+	0x4d, 0x49, 0x54, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x4f, 0x4c, 0x4c, 0x42, 0x41, 0x43,
+	0x4b, 0x10, 0x02, 0x2a, 0x57, 0x0a, 0x11, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69,
+	0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x12, 0x53, 0x4f, 0x55, 0x52,
+	0x43, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00,
+	0x12, 0x11, 0x0a, 0x0d, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e,
+	0x54, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x53, 0x45,
+	0x52, 0x56, 0x45, 0x52, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, 0x10, 0x02, 0x2a, 0x57, 0x0a, 0x0a,
+	0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x16, 0x50, 0x45,
+	0x52, 0x4d, 0x49, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49,
+	0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x01,
+	0x12, 0x08, 0x0a, 0x04, 0x52, 0x45, 0x41, 0x44, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x57, 0x52,
+	0x49, 0x54, 0x45, 0x10, 0x03, 0x12, 0x0e, 0x0a, 0x0a, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x57, 0x52,
+	0x49, 0x54, 0x45, 0x10, 0x04, 0x2a, 0x3b, 0x0a, 0x0a, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54,
+	0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x46, 0x49, 0x4c, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x59,
+	0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00,
+	0x12, 0x07, 0x0a, 0x03, 0x54, 0x41, 0x47, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x51, 0x4c,
+	0x10, 0x02, 0x2a, 0x54, 0x0a, 0x0d, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x53, 0x63, 0x68,
+	0x65, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x1a, 0x41, 0x44, 0x44, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x53,
+	0x43, 0x48, 0x45, 0x4d, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45,
+	0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x50, 0x76, 0x34, 0x10, 0x01, 0x12, 0x08, 0x0a,
+	0x04, 0x49, 0x50, 0x76, 0x36, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4f, 0x4d, 0x41, 0x49,
+	0x4e, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x03, 0x2a, 0x5d, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73,
+	0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1c, 0x0a, 0x18, 0x4d, 0x45, 0x53, 0x53, 0x41,
+	0x47, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46,
+	0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10,
+	0x01, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x49, 0x46, 0x4f, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x44,
+	0x45, 0x4c, 0x41, 0x59, 0x10, 0x03, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41,
+	0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x04, 0x2a, 0x47, 0x0a, 0x0a, 0x44, 0x69, 0x67, 0x65, 0x73,
+	0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x44, 0x49, 0x47, 0x45, 0x53, 0x54, 0x5f,
+	0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44,
+	0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x43, 0x52, 0x43, 0x33, 0x32, 0x10, 0x01, 0x12, 0x07, 0x0a,
+	0x03, 0x4d, 0x44, 0x35, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x48, 0x41, 0x31, 0x10, 0x03,
+	0x2a, 0x5f, 0x0a, 0x0a, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b,
+	0x0a, 0x17, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e,
+	0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x50,
+	0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x50, 0x55, 0x53,
+	0x48, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f,
+	0x53, 0x49, 0x4d, 0x50, 0x4c, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10,
+	0x03, 0x2a, 0x3c, 0x0a, 0x08, 0x45, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x12, 0x18, 0x0a,
+	0x14, 0x45, 0x4e, 0x43, 0x4f, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43,
+	0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x44, 0x45, 0x4e, 0x54,
+	0x49, 0x54, 0x59, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x5a, 0x49, 0x50, 0x10, 0x02, 0x2a,
+	0xa7, 0x09, 0x0a, 0x04, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x4f, 0x44, 0x45,
+	0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08,
+	0x0a, 0x02, 0x4f, 0x4b, 0x10, 0xa0, 0x9c, 0x01, 0x12, 0x16, 0x0a, 0x10, 0x4d, 0x55, 0x4c, 0x54,
+	0x49, 0x50, 0x4c, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x55, 0x4c, 0x54, 0x53, 0x10, 0xb0, 0xea, 0x01,
... 5041 lines suppressed ...