diff --git a/Makefile b/Makefile index cdae8a5983..4e119fc366 100644 --- a/Makefile +++ b/Makefile @@ -50,17 +50,23 @@ container: --build-arg PULSAR_IMAGE="${PULSAR_IMAGE}" \ --build-arg ARCH="${CONTAINER_ARCH}" . -test: container test_standalone test_clustered +test: container test_standalone test_clustered test_extensible_load_manager test_standalone: container docker run -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci.sh" test_clustered: container - PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml up -d || true + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml up -d until curl http://localhost:8080/metrics > /dev/null 2>&1; do sleep 1; done docker run --network "clustered_pulsar" -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci-clustered.sh" PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/clustered/docker-compose.yml down +test_extensible_load_manager: container + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/extensible-load-manager/docker-compose.yml up -d + until curl http://localhost:8080/metrics > /dev/null 2>&1; do sleep 1; done + docker run --network "extensible-load-manager_pulsar" -i ${IMAGE_NAME} bash -c "cd /pulsar/pulsar-client-go && ./scripts/run-ci-extensible-load-manager.sh" + PULSAR_VERSION=${PULSAR_VERSION} docker compose -f integration-tests/extensible-load-manager/docker-compose.yml down + clean: docker rmi --force $(IMAGE_NAME) || true rm bin/* diff --git a/integration-tests/extensible-load-manager/docker-compose.yml b/integration-tests/extensible-load-manager/docker-compose.yml new file mode 100644 index 0000000000..935afdcefa --- /dev/null +++ b/integration-tests/extensible-load-manager/docker-compose.yml @@ -0,0 +1,181 @@ +# 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. + +version: '3' +networks: + pulsar: + driver: bridge +services: + # Start ZooKeeper + zookeeper: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: zookeeper + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + command: > + bash -c "bin/apply-config-from-env.py conf/zookeeper.conf && \ + bin/generate-zookeeper-config.sh conf/zookeeper.conf && \ + exec bin/pulsar zookeeper" + healthcheck: + test: ["CMD", "bin/pulsar-zookeeper-ruok.sh"] + interval: 10s + timeout: 5s + retries: 30 + + # Initialize cluster metadata + pulsar-init: + container_name: pulsar-init + hostname: pulsar-init + image: apachepulsar/pulsar:${PULSAR_VERSION} + networks: + - pulsar + environment: + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + command: > + bin/pulsar initialize-cluster-metadata \ + --cluster cluster-a \ + --zookeeper zookeeper:2181 \ + --configuration-store zookeeper:2181 \ + --web-service-url http://broker-1:8080 \ + --broker-service-url pulsar://broker-1:6650 + depends_on: + zookeeper: + condition: service_healthy + + # Start bookie + bookie: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: bookie + restart: on-failure + networks: + - pulsar + environment: + - clusterName=cluster-a + - zkServers=zookeeper:2181 + - metadataServiceUri=metadata-store:zk:zookeeper:2181 + - advertisedAddress=bookie + - BOOKIE_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + depends_on: + zookeeper: + condition: service_healthy + pulsar-init: + condition: service_completed_successfully + command: bash -c "bin/apply-config-from-env.py conf/bookkeeper.conf && exec bin/pulsar bookie" + + proxy: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: proxy + hostname: proxy + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - PULSAR_MEM=-Xms128m -Xmx128m -XX:MaxDirectMemorySize=56m + ports: + - "8080:8080" + - "6650:6650" + depends_on: + broker-1: + condition: service_healthy + broker-2: + condition: service_healthy + command: bash -c "bin/apply-config-from-env.py conf/proxy.conf && exec bin/pulsar proxy" + + # Start broker 1 + broker-1: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: broker-1 + hostname: broker-1 + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - managedLedgerDefaultEnsembleSize=1 + - managedLedgerDefaultWriteQuorum=1 + - managedLedgerDefaultAckQuorum=1 + - advertisedAddress=broker-1 + - internalListenerName=internal + - advertisedListeners=internal:pulsar://broker-1:6650 + - PULSAR_MEM=-Xms256m -Xmx256m -XX:MaxDirectMemorySize=56m + # Load Manager. Here uses the extensible load balancer, sets the unloading strategy to TransferShedder, and enables debug mode. + - loadManagerClassName=org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl + - loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder + - loadBalancerSheddingEnabled=false + - loadBalancerDebugModeEnabled=true + - clusterMigrationCheckDurationSeconds=1 + - brokerServiceCompactionThresholdInBytes=1000000 + - PULSAR_PREFIX_defaultNumberOfNamespaceBundles=1 + depends_on: + zookeeper: + condition: service_healthy + bookie: + condition: service_started + command: bash -c "bin/apply-config-from-env.py conf/broker.conf && exec bin/pulsar broker" + healthcheck: + test: ["CMD-SHELL", "curl -f http://localhost:8080/metrics || exit 1"] + interval: 10s + timeout: 5s + retries: 30 + + # Start broker 2 + broker-2: + image: apachepulsar/pulsar:${PULSAR_VERSION} + container_name: broker-2 + hostname: broker-2 + restart: on-failure + networks: + - pulsar + environment: + - metadataStoreUrl=zk:zookeeper:2181 + - zookeeperServers=zookeeper:2181 + - clusterName=cluster-a + - managedLedgerDefaultEnsembleSize=1 + - managedLedgerDefaultWriteQuorum=1 + - managedLedgerDefaultAckQuorum=1 + - advertisedAddress=broker-2 + - internalListenerName=internal + - advertisedListeners=internal:pulsar://broker-2:6650 + - PULSAR_MEM=-Xms256m -Xmx256m -XX:MaxDirectMemorySize=56m + # Load Manager. Here uses the extensible load balancer, sets the unloading strategy to TransferShedder, and enables debug mode. + - loadManagerClassName=org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl + - loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder + - loadBalancerSheddingEnabled=false + - loadBalancerDebugModeEnabled=true + - clusterMigrationCheckDurationSeconds=1 + - brokerServiceCompactionThresholdInBytes=1000000 + - PULSAR_PREFIX_defaultNumberOfNamespaceBundles=1 + depends_on: + zookeeper: + condition: service_healthy + bookie: + condition: service_started + command: bash -c "bin/apply-config-from-env.py conf/broker.conf && exec bin/pulsar broker" + healthcheck: + test: ["CMD-SHELL", "curl -f http://localhost:8080/metrics || exit 1"] + interval: 10s + timeout: 5s + retries: 30 diff --git a/pulsar/client_impl.go b/pulsar/client_impl.go index 65aed3b963..8471e1e39a 100644 --- a/pulsar/client_impl.go +++ b/pulsar/client_impl.go @@ -49,6 +49,7 @@ type client struct { memLimit internal.MemoryLimitController closeOnce sync.Once operationTimeout time.Duration + tlsEnabled bool log log.Logger } @@ -166,6 +167,7 @@ func newClient(options ClientOptions) (Client, error) { metrics: metrics, memLimit: internal.NewMemoryLimitController(memLimitBytes, defaultMemoryLimitTriggerThreshold), operationTimeout: operationTimeout, + tlsEnabled: tlsConfig != nil, } serviceNameResolver := internal.NewPulsarServiceNameResolver(url) @@ -174,7 +176,7 @@ func newClient(options ClientOptions) (Client, error) { switch url.Scheme { case "pulsar", "pulsar+ssl": c.lookupService = internal.NewLookupService(c.rpcClient, url, serviceNameResolver, - tlsConfig != nil, options.ListenerName, logger, metrics) + c.tlsEnabled, options.ListenerName, logger, metrics) case "http", "https": httpClient, err := internal.NewHTTPClient(url, serviceNameResolver, tlsConfig, operationTimeout, logger, metrics, authProvider) @@ -183,7 +185,7 @@ func newClient(options ClientOptions) (Client, error) { err.Error())) } c.lookupService = internal.NewHTTPLookupService(httpClient, url, serviceNameResolver, - tlsConfig != nil, logger, metrics) + c.tlsEnabled, logger, metrics) default: return nil, newError(InvalidConfiguration, fmt.Sprintf("Invalid URL scheme '%s'", url.Scheme)) } @@ -275,3 +277,10 @@ func (c *client) Close() { c.lookupService.Close() }) } + +func (c *client) selectServiceURL(brokerServiceURL, brokerServiceURLTLS string) string { + if c.tlsEnabled { + return brokerServiceURLTLS + } + return brokerServiceURL +} diff --git a/pulsar/consumer_partition.go b/pulsar/consumer_partition.go index 162565b2a9..dc01e69285 100644 --- a/pulsar/consumer_partition.go +++ b/pulsar/consumer_partition.go @@ -158,7 +158,7 @@ type partitionConsumer struct { eventsCh chan interface{} connectedCh chan struct{} - connectClosedCh chan connectionClosed + connectClosedCh chan *connectionClosed closeCh chan struct{} clearQueueCh chan func(id *trackingMessageID) @@ -326,7 +326,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon startMessageID: atomicMessageID{msgID: options.startMessageID}, connectedCh: make(chan struct{}), messageCh: messageCh, - connectClosedCh: make(chan connectionClosed, 10), + connectClosedCh: make(chan *connectionClosed, 10), closeCh: make(chan struct{}), clearQueueCh: make(chan func(id *trackingMessageID)), compressionProviders: sync.Map{}, @@ -370,7 +370,7 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon pc.nackTracker = newNegativeAcksTracker(pc, options.nackRedeliveryDelay, options.nackBackoffPolicy, pc.log) - err := pc.grabConn() + err := pc.grabConn("") if err != nil { pc.log.WithError(err).Error("Failed to create consumer") pc.nackTracker.Close() @@ -1358,10 +1358,17 @@ func createEncryptionContext(msgMeta *pb.MessageMetadata) *EncryptionContext { return &encCtx } -func (pc *partitionConsumer) ConnectionClosed() { +func (pc *partitionConsumer) ConnectionClosed(closeConsumer *pb.CommandCloseConsumer) { // Trigger reconnection in the consumer goroutine pc.log.Debug("connection closed and send to connectClosedCh") - pc.connectClosedCh <- connectionClosed{} + var assignedBrokerURL string + if closeConsumer != nil { + assignedBrokerURL = pc.client.selectServiceURL( + closeConsumer.GetAssignedBrokerServiceUrl(), closeConsumer.GetAssignedBrokerServiceUrlTls()) + } + pc.connectClosedCh <- &connectionClosed{ + assignedBrokerURL: assignedBrokerURL, + } } // Flow command gives additional permits to send messages to the consumer. @@ -1566,9 +1573,9 @@ func (pc *partitionConsumer) runEventsLoop() { case <-pc.closeCh: pc.log.Info("close consumer, exit reconnect") return - case <-pc.connectClosedCh: + case connectionClosed := <-pc.connectClosedCh: pc.log.Debug("runEventsLoop will reconnect") - pc.reconnectToBroker() + pc.reconnectToBroker(connectionClosed) } } }() @@ -1652,7 +1659,7 @@ func (pc *partitionConsumer) internalClose(req *closeRequest) { close(pc.closeCh) } -func (pc *partitionConsumer) reconnectToBroker() { +func (pc *partitionConsumer) reconnectToBroker(connectionClosed *connectionClosed) { var maxRetry int if pc.options.maxReconnectToBroker == nil { @@ -1673,13 +1680,22 @@ func (pc *partitionConsumer) reconnectToBroker() { return } - if pc.options.backoffPolicy == nil { + var assignedBrokerURL string + + if connectionClosed != nil && connectionClosed.HasURL() { + delayReconnectTime = 0 + assignedBrokerURL = connectionClosed.assignedBrokerURL + connectionClosed = nil // Attempt connecting to the assigned broker just once + } else if pc.options.backoffPolicy == nil { delayReconnectTime = defaultBackoff.Next() } else { delayReconnectTime = pc.options.backoffPolicy.Next() } - pc.log.Info("Reconnecting to broker in ", delayReconnectTime) + pc.log.WithFields(log.Fields{ + "assignedBrokerURL": assignedBrokerURL, + "delayReconnectTime": delayReconnectTime, + }).Info("Reconnecting to broker") time.Sleep(delayReconnectTime) // double check @@ -1689,7 +1705,7 @@ func (pc *partitionConsumer) reconnectToBroker() { return } - err := pc.grabConn() + err := pc.grabConn(assignedBrokerURL) if err == nil { // Successfully reconnected pc.log.Info("Reconnected consumer to broker") @@ -1713,13 +1729,25 @@ func (pc *partitionConsumer) reconnectToBroker() { } } -func (pc *partitionConsumer) grabConn() error { - lr, err := pc.client.lookupService.Lookup(pc.topic) +func (pc *partitionConsumer) lookupTopic(brokerServiceURL string) (*internal.LookupResult, error) { + if len(brokerServiceURL) == 0 { + lr, err := pc.client.lookupService.Lookup(pc.topic) + if err != nil { + pc.log.WithError(err).Warn("Failed to lookup topic") + return nil, err + } + + pc.log.Debug("Lookup result: ", lr) + return lr, err + } + return pc.client.lookupService.GetBrokerAddress(brokerServiceURL, pc._getConn().IsProxied()) +} + +func (pc *partitionConsumer) grabConn(assignedBrokerURL string) error { + lr, err := pc.lookupTopic(assignedBrokerURL) if err != nil { - pc.log.WithError(err).Warn("Failed to lookup topic") return err } - pc.log.Debugf("Lookup result: %+v", lr) subType := toProtoSubType(pc.options.subscriptionType) initialPosition := toProtoInitialPosition(pc.options.subscriptionInitPos) diff --git a/pulsar/consumer_test.go b/pulsar/consumer_test.go index 4a3b532d05..4120ba4bac 100644 --- a/pulsar/consumer_test.go +++ b/pulsar/consumer_test.go @@ -3796,22 +3796,22 @@ func TestConsumerWithBackoffPolicy(t *testing.T) { partitionConsumerImp := _consumer.(*consumer).consumers[0] // 1 s startTime := time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 2 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } diff --git a/pulsar/extensible_load_manager_test.go b/pulsar/extensible_load_manager_test.go new file mode 100644 index 0000000000..b3025c390a --- /dev/null +++ b/pulsar/extensible_load_manager_test.go @@ -0,0 +1,231 @@ +//go:build extensible_load_manager + +// 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 pulsar + +import ( + "context" + "fmt" + "net/http" + "sync" + "testing" + "time" + + "github.com/apache/pulsar-client-go/pulsaradmin" + "github.com/apache/pulsar-client-go/pulsaradmin/pkg/utils" + + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/suite" + + uAtomic "go.uber.org/atomic" +) + +type ExtensibleLoadManagerTestSuite struct { + suite.Suite +} + +func TestExtensibleLoadManagerTestSuite(t *testing.T) { + suite.Run(t, new(ExtensibleLoadManagerTestSuite)) +} + +const ( + tenant = utils.PUBLICTENANT + namespace = utils.DEFAULTNAMESPACE + + broker1URL = "pulsar://broker-1:6650" + broker2URL = "pulsar://broker-2:6650" + + broker1LookupURL = "broker-1:8080" + broker2LookupURL = "broker-2:8080" +) + +type mockCounter struct { + prometheus.Counter + + count uAtomic.Int32 +} + +func (m *mockCounter) Inc() { + m.count.Inc() +} + +func (suite *ExtensibleLoadManagerTestSuite) TestTopicUnload() { + type topicUnloadTestCase struct { + testCaseName string + adminURL string + clientEndpointFunc func(utils.LookupData) string + unloadEndpointFunc func(utils.LookupData) string + } + for _, scenario := range []topicUnloadTestCase{ + { + testCaseName: "directConnection", + adminURL: "http://broker-1:8080", + clientEndpointFunc: func(lookupResult utils.LookupData) string { + return lookupResult.BrokerURL + }, + unloadEndpointFunc: func(lookupResult utils.LookupData) string { + return lookupResult.HTTPURL + }, + }, + { + testCaseName: "proxyConnection", + adminURL: "http://proxy:8080", + clientEndpointFunc: func(utils.LookupData) string { + return "pulsar://proxy:6650" + }, + unloadEndpointFunc: func(utils.LookupData) string { + return "http://proxy:8080" + }, + }, + } { + suite.T().Run(scenario.testCaseName, func(t *testing.T) { + testTopicUnload(t, scenario.adminURL, scenario.clientEndpointFunc, scenario.unloadEndpointFunc) + }) + } +} + +func testTopicUnload(t *testing.T, adminURL string, + clientEndpointFunc func(utils.LookupData) string, + unloadEndpointFunc func(utils.LookupData) string) { + req := assert.New(t) + + admin, err := pulsaradmin.NewClient(&pulsaradmin.Config{WebServiceURL: adminURL}) + req.NoError(err) + + topicName, err := utils.GetTopicName(newTopicName()) + req.NoError(err) + req.NotNil(topicName) + + err = admin.Topics().Create(*topicName, 0) + req.NoError(err) + + lookupResult, err := admin.Topics().Lookup(*topicName) + req.NoError(err) + req.NotEmpty(lookupResult.BrokerURL) + srcTopicBrokerURL := lookupResult.BrokerURL + req.Contains([...]string{broker1URL, broker2URL}, srcTopicBrokerURL) + + var dstTopicBrokerURL string + if srcTopicBrokerURL == broker1URL { + dstTopicBrokerURL = broker2LookupURL + } else { + dstTopicBrokerURL = broker1LookupURL + } + + bundleRange, err := admin.Topics().GetBundleRange(*topicName) + req.NoError(err) + req.NotEmpty(bundleRange) + + clientURL := clientEndpointFunc(lookupResult) + pulsarClient, err := NewClient(ClientOptions{URL: clientURL}) + req.NoError(err) + defer pulsarClient.Close() + + producer, err := pulsarClient.CreateProducer(ProducerOptions{ + Topic: topicName.String(), + }) + req.NoError(err) + defer producer.Close() + + consumer, err := pulsarClient.Subscribe(ConsumerOptions{ + Topic: topicName.String(), + SubscriptionName: fmt.Sprintf("my-sub-%v", time.Now().Nanosecond()), + SubscriptionInitialPosition: SubscriptionPositionEarliest, + }) + req.NoError(err) + defer consumer.Close() + + pulsarClientImpl := pulsarClient.(*client) + lookupRequestCounterMock := mockCounter{} + pulsarClientImpl.metrics.LookupRequestsCount = &lookupRequestCounterMock + + messageCountBeforeUnload := 100 + messageCountDuringUnload := 100 + messageCountAfterUnload := 100 + messageCount := messageCountBeforeUnload + messageCountDuringUnload + messageCountAfterUnload + + // Signals all goroutines have completed + wgRoutines := sync.WaitGroup{} + wgRoutines.Add(2) + + // Signals unload has completed + wgUnload := sync.WaitGroup{} + wgUnload.Add(1) + + // Signals both producer and consumer have processed `messageCountBeforeUnload` messages + wgSendAndReceiveMessages := sync.WaitGroup{} + wgSendAndReceiveMessages.Add(2) + + // Producer + go func() { + defer wgRoutines.Done() + + for i := 0; i < messageCount; i++ { + if i == messageCountBeforeUnload+messageCountDuringUnload { + wgUnload.Wait() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + + pm := ProducerMessage{Payload: []byte(fmt.Sprintf("hello-%d", i))} + _, err := producer.Send(ctx, &pm) + req.NoError(err) + req.NoError(ctx.Err()) + + if i == messageCountBeforeUnload { + wgSendAndReceiveMessages.Done() + } + } + }() + + // Consumer + go func() { + defer wgRoutines.Done() + + for i := 0; i < messageCount; i++ { + if i == messageCountBeforeUnload+messageCountDuringUnload { + wgUnload.Wait() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + + _, err := consumer.Receive(ctx) + req.NoError(err) + req.NoError(ctx.Err()) + + if i == messageCountBeforeUnload { + wgSendAndReceiveMessages.Done() + } + } + }() + + // Unload the bundle, triggering the producers and consumers to reconnect to the specified broker. + wgSendAndReceiveMessages.Wait() + unloadEndpoint := unloadEndpointFunc(lookupResult) + unloadURL := fmt.Sprintf( + "/admin/v2/namespaces/%s/%s/%s/unload?destinationBroker=%s", tenant, namespace, bundleRange, dstTopicBrokerURL) + makeHTTPCall(t, http.MethodPut, unloadEndpoint+unloadURL, "") + wgUnload.Done() + + wgRoutines.Wait() + req.Equal(int32(0), lookupRequestCounterMock.count.Load()) +} diff --git a/pulsar/internal/connection.go b/pulsar/internal/connection.go index cb8c680d48..9244328bc2 100644 --- a/pulsar/internal/connection.go +++ b/pulsar/internal/connection.go @@ -69,7 +69,7 @@ type ConnectionListener interface { ReceivedSendReceipt(response *pb.CommandSendReceipt) // ConnectionClosed close the TCP connection. - ConnectionClosed() + ConnectionClosed(closeProducer *pb.CommandCloseProducer) } // Connection is a interface of client cnx. @@ -84,6 +84,7 @@ type Connection interface { ID() string GetMaxMessageSize() int32 Close() + IsProxied() bool } type ConsumerHandler interface { @@ -92,7 +93,7 @@ type ConsumerHandler interface { ActiveConsumerChanged(isActive bool) // ConnectionClosed close the TCP connection. - ConnectionClosed() + ConnectionClosed(closeConsumer *pb.CommandCloseConsumer) } type connectionState int32 @@ -312,7 +313,7 @@ func (c *connection) doHandshake() bool { }, } - if c.logicalAddr.Host != c.physicalAddr.Host { + if c.IsProxied() { cmdConnect.ProxyToBrokerUrl = proto.String(c.logicalAddr.Host) } c.writeCommand(baseCommand(pb.BaseCommand_CONNECT, cmdConnect)) @@ -343,6 +344,10 @@ func (c *connection) doHandshake() bool { return true } +func (c *connection) IsProxied() bool { + return c.logicalAddr.Host != c.physicalAddr.Host +} + func (c *connection) waitUntilReady() error { // If we are going to call cond.Wait() at all, then we must call it _before_ we call cond.Broadcast(). // The lock is held here to prevent changeState() from calling cond.Broadcast() in the time between @@ -893,7 +898,7 @@ func (c *connection) handleCloseConsumer(closeConsumer *pb.CommandCloseConsumer) c.log.Infof("Broker notification of Closed consumer: %d", consumerID) if consumer, ok := c.consumerHandler(consumerID); ok { - consumer.ConnectionClosed() + consumer.ConnectionClosed(closeConsumer) c.DeleteConsumeHandler(consumerID) } else { c.log.WithField("consumerID", consumerID).Warnf("Consumer with ID not found while closing consumer") @@ -917,7 +922,7 @@ func (c *connection) handleCloseProducer(closeProducer *pb.CommandCloseProducer) producer, ok := c.deletePendingProducers(producerID) // did we find a producer? if ok { - producer.ConnectionClosed() + producer.ConnectionClosed(closeProducer) } else { c.log.WithField("producerID", producerID).Warn("Producer with ID not found while closing producer") } @@ -1024,12 +1029,12 @@ func (c *connection) Close() { // notify producers connection closed for _, listener := range listeners { - listener.ConnectionClosed() + listener.ConnectionClosed(nil) } // notify consumers connection closed for _, handler := range consumerHandlers { - handler.ConnectionClosed() + handler.ConnectionClosed(nil) } c.metrics.ConnectionsClosed.Inc() diff --git a/pulsar/internal/lookup_service.go b/pulsar/internal/lookup_service.go index e30bec1d73..8b7fb01b2f 100644 --- a/pulsar/internal/lookup_service.go +++ b/pulsar/internal/lookup_service.go @@ -64,6 +64,8 @@ type LookupService interface { // GetSchema returns schema for a given version. GetSchema(topic string, schemaVersion []byte) (schema *pb.Schema, err error) + GetBrokerAddress(brokerServiceURL string, proxyThroughServiceURL bool) (*LookupResult, error) + // Closable Allow Lookup Service's internal client to be able to closed Closable } @@ -107,29 +109,26 @@ func (ls *lookupService) GetSchema(topic string, schemaVersion []byte) (schema * return res.Response.GetSchemaResponse.Schema, nil } -func (ls *lookupService) getBrokerAddress(lr *pb.CommandLookupTopicResponse) (logicalAddress *url.URL, - physicalAddress *url.URL, err error) { - if ls.tlsEnabled { - logicalAddress, err = url.ParseRequestURI(lr.GetBrokerServiceUrlTls()) - } else { - logicalAddress, err = url.ParseRequestURI(lr.GetBrokerServiceUrl()) - } - +func (ls *lookupService) GetBrokerAddress(brokerServiceURL string, proxyThroughServiceURL bool) (*LookupResult, error) { + logicalAddress, err := url.ParseRequestURI(brokerServiceURL) if err != nil { - return nil, nil, err + return nil, err } - var physicalAddr *url.URL - if lr.GetProxyThroughServiceUrl() { - physicalAddr, err = ls.serviceNameResolver.ResolveHost() + var physicalAddress *url.URL + if proxyThroughServiceURL { + physicalAddress, err = ls.serviceNameResolver.ResolveHost() if err != nil { - return nil, nil, err + return nil, err } } else { - physicalAddr = logicalAddress + physicalAddress = logicalAddress } - return logicalAddress, physicalAddr, nil + return &LookupResult{ + LogicalAddr: logicalAddress, + PhysicalAddr: physicalAddress, + }, nil } // Follow brokers redirect up to certain number of times @@ -154,7 +153,8 @@ func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { switch *lr.Response { case pb.CommandLookupTopicResponse_Redirect: - logicalAddress, physicalAddr, err := ls.getBrokerAddress(lr) + brokerServiceURL := selectServiceURL(ls.tlsEnabled, lr.GetBrokerServiceUrl(), lr.GetBrokerServiceUrlTls()) + lookupResult, err := ls.GetBrokerAddress(brokerServiceURL, lr.GetProxyThroughServiceUrl()) if err != nil { return nil, err } @@ -163,12 +163,13 @@ func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { topic, lr.BrokerServiceUrl, lr.BrokerServiceUrlTls, lr.ProxyThroughServiceUrl) id := ls.rpcClient.NewRequestID() - res, err = ls.rpcClient.Request(logicalAddress, physicalAddr, id, pb.BaseCommand_LOOKUP, &pb.CommandLookupTopic{ - RequestId: &id, - Topic: &topic, - Authoritative: lr.Authoritative, - AdvertisedListenerName: proto.String(ls.listenerName), - }) + res, err = ls.rpcClient.Request(lookupResult.LogicalAddr, lookupResult.PhysicalAddr, id, pb.BaseCommand_LOOKUP, + &pb.CommandLookupTopic{ + RequestId: &id, + Topic: &topic, + Authoritative: lr.Authoritative, + AdvertisedListenerName: proto.String(ls.listenerName), + }) if err != nil { return nil, err } @@ -180,16 +181,8 @@ func (ls *lookupService) Lookup(topic string) (*LookupResult, error) { ls.log.Debugf("Successfully looked up topic{%s} on broker. %s / %s - Use proxy: %t", topic, lr.GetBrokerServiceUrl(), lr.GetBrokerServiceUrlTls(), lr.GetProxyThroughServiceUrl()) - logicalAddress, physicalAddress, err := ls.getBrokerAddress(lr) - if err != nil { - return nil, err - } - - return &LookupResult{ - LogicalAddr: logicalAddress, - PhysicalAddr: physicalAddress, - }, nil - + brokerServiceURL := selectServiceURL(ls.tlsEnabled, lr.GetBrokerServiceUrl(), lr.GetBrokerServiceUrlTls()) + return ls.GetBrokerAddress(brokerServiceURL, lr.GetProxyThroughServiceUrl()) case pb.CommandLookupTopicResponse_Failed: ls.log.WithFields(log.Fields{ "topic": topic, @@ -284,19 +277,15 @@ type httpLookupService struct { metrics *Metrics } -func (h *httpLookupService) getBrokerAddress(ld *httpLookupData) (logicalAddress *url.URL, - physicalAddress *url.URL, err error) { - if h.tlsEnabled { - logicalAddress, err = url.ParseRequestURI(ld.BrokerURLTLS) - } else { - logicalAddress, err = url.ParseRequestURI(ld.BrokerURL) - } - +func (h *httpLookupService) GetBrokerAddress(brokerServiceURL string, _ bool) (*LookupResult, error) { + logicalAddress, err := url.ParseRequestURI(brokerServiceURL) if err != nil { - return nil, nil, err + return nil, err } - - return logicalAddress, logicalAddress, nil + return &LookupResult{ + LogicalAddr: logicalAddress, + PhysicalAddr: logicalAddress, + }, err } func (h *httpLookupService) Lookup(topic string) (*LookupResult, error) { @@ -319,16 +308,8 @@ func (h *httpLookupService) Lookup(topic string) (*LookupResult, error) { h.log.Debugf("Successfully looked up topic{%s} on http broker. %+v", topic, lookupData) - logicalAddress, physicalAddress, err := h.getBrokerAddress(lookupData) - if err != nil { - return nil, err - } - - return &LookupResult{ - LogicalAddr: logicalAddress, - PhysicalAddr: physicalAddress, - }, nil - + brokerServiceURL := selectServiceURL(h.tlsEnabled, lookupData.BrokerURL, lookupData.BrokerURLTLS) + return h.GetBrokerAddress(brokerServiceURL, false /* ignored */) } func (h *httpLookupService) GetPartitionedTopicMetadata(topic string) (*PartitionedTopicMetadata, @@ -397,3 +378,10 @@ func NewHTTPLookupService(httpClient HTTPClient, serviceURL *url.URL, serviceNam metrics: metrics, } } + +func selectServiceURL(tlsEnabled bool, brokerServiceURL, brokerServiceURLTLS string) string { + if tlsEnabled { + return brokerServiceURLTLS + } + return brokerServiceURL +} diff --git a/pulsar/internal/pulsar_proto/PulsarApi.pb.go b/pulsar/internal/pulsar_proto/PulsarApi.pb.go index c8e6ad9b91..5c6e0f3ef5 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.pb.go +++ b/pulsar/internal/pulsar_proto/PulsarApi.pb.go @@ -1,4 +1,4 @@ -//* +// // 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 @@ -19,7 +19,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.28.1 -// protoc v3.21.9 +// protoc v3.21.12 // source: PulsarApi.proto package proto @@ -1744,7 +1744,7 @@ type MessageMetadata struct { // Property set on replicated message, // includes the source cluster name ReplicatedFrom *string `protobuf:"bytes,5,opt,name=replicated_from,json=replicatedFrom" json:"replicated_from,omitempty"` - //key to decide partition for the msg + // key to decide partition for the msg PartitionKey *string `protobuf:"bytes,6,opt,name=partition_key,json=partitionKey" json:"partition_key,omitempty"` // Override namespace's replication ReplicateTo []string `protobuf:"bytes,7,rep,name=replicate_to,json=replicateTo" json:"replicate_to,omitempty"` @@ -1752,7 +1752,7 @@ type MessageMetadata struct { UncompressedSize *uint32 `protobuf:"varint,9,opt,name=uncompressed_size,json=uncompressedSize,def=0" json:"uncompressed_size,omitempty"` // Removed below checksum field from Metadata as // it should be part of send-command which keeps checksum of header + payload - //optional sfixed64 checksum = 10; + // optional sfixed64 checksum = 10; // differentiate single and batch message metadata NumMessagesInBatch *int32 `protobuf:"varint,11,opt,name=num_messages_in_batch,json=numMessagesInBatch,def=1" json:"num_messages_in_batch,omitempty"` // the timestamp that this event occurs. it is typically set by applications. @@ -1777,7 +1777,7 @@ type MessageMetadata struct { // transaction related message info TxnidLeastBits *uint64 `protobuf:"varint,22,opt,name=txnid_least_bits,json=txnidLeastBits" json:"txnid_least_bits,omitempty"` TxnidMostBits *uint64 `protobuf:"varint,23,opt,name=txnid_most_bits,json=txnidMostBits" json:"txnid_most_bits,omitempty"` - /// Add highest sequence id to support batch message with external sequence id + // / Add highest sequence id to support batch message with external sequence id HighestSequenceId *uint64 `protobuf:"varint,24,opt,name=highest_sequence_id,json=highestSequenceId,def=0" json:"highest_sequence_id,omitempty"` // Indicate if the message payload value is set NullValue *bool `protobuf:"varint,25,opt,name=null_value,json=nullValue,def=0" json:"null_value,omitempty"` @@ -2786,7 +2786,7 @@ type CommandSubscribe struct { // markd-delete position on the particular message id and // will send messages from that point StartMessageId *MessageIdData `protobuf:"bytes,9,opt,name=start_message_id,json=startMessageId" json:"start_message_id,omitempty"` - /// Add optional metadata key=value to this consumer + // / Add optional metadata key=value to this consumer Metadata []*KeyValue `protobuf:"bytes,10,rep,name=metadata" json:"metadata,omitempty"` ReadCompacted *bool `protobuf:"varint,11,opt,name=read_compacted,json=readCompacted" json:"read_compacted,omitempty"` Schema *Schema `protobuf:"bytes,12,opt,name=schema" json:"schema,omitempty"` @@ -3164,7 +3164,6 @@ type CommandLookupTopic struct { // to the proxy. OriginalAuthData *string `protobuf:"bytes,5,opt,name=original_auth_data,json=originalAuthData" json:"original_auth_data,omitempty"` OriginalAuthMethod *string `protobuf:"bytes,6,opt,name=original_auth_method,json=originalAuthMethod" json:"original_auth_method,omitempty"` - // AdvertisedListenerName *string `protobuf:"bytes,7,opt,name=advertised_listener_name,json=advertisedListenerName" json:"advertised_listener_name,omitempty"` } @@ -3366,8 +3365,8 @@ func (x *CommandLookupTopicResponse) GetProxyThroughServiceUrl() bool { return Default_CommandLookupTopicResponse_ProxyThroughServiceUrl } -/// Create a new Producer on a topic, assigning the given producer_id, -/// all messages sent with this producer_id will be persisted on the topic +// / Create a new Producer on a topic, assigning the given producer_id, +// / all messages sent with this producer_id will be persisted on the topic type CommandProducer struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3376,11 +3375,11 @@ type CommandProducer struct { Topic *string `protobuf:"bytes,1,req,name=topic" json:"topic,omitempty"` ProducerId *uint64 `protobuf:"varint,2,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` RequestId *uint64 `protobuf:"varint,3,req,name=request_id,json=requestId" json:"request_id,omitempty"` - /// If a producer name is specified, the name will be used, - /// otherwise the broker will generate a unique name + // / If a producer name is specified, the name will be used, + // / otherwise the broker will generate a unique name ProducerName *string `protobuf:"bytes,4,opt,name=producer_name,json=producerName" json:"producer_name,omitempty"` Encrypted *bool `protobuf:"varint,5,opt,name=encrypted,def=0" json:"encrypted,omitempty"` - /// Add optional metadata key=value to this producer + // / Add optional metadata key=value to this producer Metadata []*KeyValue `protobuf:"bytes,6,rep,name=metadata" json:"metadata,omitempty"` Schema *Schema `protobuf:"bytes,7,opt,name=schema" json:"schema,omitempty"` // If producer reconnect to broker, the epoch of this producer will +1 @@ -3546,7 +3545,7 @@ type CommandSend struct { NumMessages *int32 `protobuf:"varint,3,opt,name=num_messages,json=numMessages,def=1" json:"num_messages,omitempty"` TxnidLeastBits *uint64 `protobuf:"varint,4,opt,name=txnid_least_bits,json=txnidLeastBits,def=0" json:"txnid_least_bits,omitempty"` TxnidMostBits *uint64 `protobuf:"varint,5,opt,name=txnid_most_bits,json=txnidMostBits,def=0" json:"txnid_most_bits,omitempty"` - /// Add highest sequence id to support batch message with external sequence id + // / Add highest sequence id to support batch message with external sequence id HighestSequenceId *uint64 `protobuf:"varint,6,opt,name=highest_sequence_id,json=highestSequenceId,def=0" json:"highest_sequence_id,omitempty"` IsChunk *bool `protobuf:"varint,7,opt,name=is_chunk,json=isChunk,def=0" json:"is_chunk,omitempty"` // Specify if the message being published is a Pulsar marker or not @@ -4394,8 +4393,10 @@ type CommandCloseProducer struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ProducerId *uint64 `protobuf:"varint,1,req,name=producer_id,json=producerId" json:"producer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` + AssignedBrokerServiceUrl *string `protobuf:"bytes,3,opt,name=assignedBrokerServiceUrl" json:"assignedBrokerServiceUrl,omitempty"` + AssignedBrokerServiceUrlTls *string `protobuf:"bytes,4,opt,name=assignedBrokerServiceUrlTls" json:"assignedBrokerServiceUrlTls,omitempty"` } func (x *CommandCloseProducer) Reset() { @@ -4444,13 +4445,29 @@ func (x *CommandCloseProducer) GetRequestId() uint64 { return 0 } +func (x *CommandCloseProducer) GetAssignedBrokerServiceUrl() string { + if x != nil && x.AssignedBrokerServiceUrl != nil { + return *x.AssignedBrokerServiceUrl + } + return "" +} + +func (x *CommandCloseProducer) GetAssignedBrokerServiceUrlTls() string { + if x != nil && x.AssignedBrokerServiceUrlTls != nil { + return *x.AssignedBrokerServiceUrlTls + } + return "" +} + type CommandCloseConsumer struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` - RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` + ConsumerId *uint64 `protobuf:"varint,1,req,name=consumer_id,json=consumerId" json:"consumer_id,omitempty"` + RequestId *uint64 `protobuf:"varint,2,req,name=request_id,json=requestId" json:"request_id,omitempty"` + AssignedBrokerServiceUrl *string `protobuf:"bytes,3,opt,name=assignedBrokerServiceUrl" json:"assignedBrokerServiceUrl,omitempty"` + AssignedBrokerServiceUrlTls *string `protobuf:"bytes,4,opt,name=assignedBrokerServiceUrlTls" json:"assignedBrokerServiceUrlTls,omitempty"` } func (x *CommandCloseConsumer) Reset() { @@ -4499,6 +4516,20 @@ func (x *CommandCloseConsumer) GetRequestId() uint64 { return 0 } +func (x *CommandCloseConsumer) GetAssignedBrokerServiceUrl() string { + if x != nil && x.AssignedBrokerServiceUrl != nil { + return *x.AssignedBrokerServiceUrl + } + return "" +} + +func (x *CommandCloseConsumer) GetAssignedBrokerServiceUrlTls() string { + if x != nil && x.AssignedBrokerServiceUrlTls != nil { + return *x.AssignedBrokerServiceUrlTls + } + return "" +} + type CommandRedeliverUnacknowledgedMessages struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4617,7 +4648,7 @@ func (x *CommandSuccess) GetSchema() *Schema { return nil } -/// Response from CommandProducer +// / Response from CommandProducer type CommandProducerSuccess struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4925,31 +4956,31 @@ type CommandConsumerStatsResponse struct { RequestId *uint64 `protobuf:"varint,1,req,name=request_id,json=requestId" json:"request_id,omitempty"` ErrorCode *ServerError `protobuf:"varint,2,opt,name=error_code,json=errorCode,enum=pulsar.proto.ServerError" json:"error_code,omitempty"` ErrorMessage *string `protobuf:"bytes,3,opt,name=error_message,json=errorMessage" json:"error_message,omitempty"` - /// Total rate of messages delivered to the consumer. msg/s + // / Total rate of messages delivered to the consumer. msg/s MsgRateOut *float64 `protobuf:"fixed64,4,opt,name=msgRateOut" json:"msgRateOut,omitempty"` - /// Total throughput delivered to the consumer. bytes/s + // / Total throughput delivered to the consumer. bytes/s MsgThroughputOut *float64 `protobuf:"fixed64,5,opt,name=msgThroughputOut" json:"msgThroughputOut,omitempty"` - /// Total rate of messages redelivered by this consumer. msg/s + // / Total rate of messages redelivered by this consumer. msg/s MsgRateRedeliver *float64 `protobuf:"fixed64,6,opt,name=msgRateRedeliver" json:"msgRateRedeliver,omitempty"` - /// Name of the consumer + // / Name of the consumer ConsumerName *string `protobuf:"bytes,7,opt,name=consumerName" json:"consumerName,omitempty"` - /// Number of available message permits for the consumer + // / Number of available message permits for the consumer AvailablePermits *uint64 `protobuf:"varint,8,opt,name=availablePermits" json:"availablePermits,omitempty"` - /// Number of unacknowledged messages for the consumer + // / Number of unacknowledged messages for the consumer UnackedMessages *uint64 `protobuf:"varint,9,opt,name=unackedMessages" json:"unackedMessages,omitempty"` - /// Flag to verify if consumer is blocked due to reaching threshold of unacked messages + // / Flag to verify if consumer is blocked due to reaching threshold of unacked messages BlockedConsumerOnUnackedMsgs *bool `protobuf:"varint,10,opt,name=blockedConsumerOnUnackedMsgs" json:"blockedConsumerOnUnackedMsgs,omitempty"` - /// Address of this consumer + // / Address of this consumer Address *string `protobuf:"bytes,11,opt,name=address" json:"address,omitempty"` - /// Timestamp of connection + // / Timestamp of connection ConnectedSince *string `protobuf:"bytes,12,opt,name=connectedSince" json:"connectedSince,omitempty"` - /// Whether this subscription is Exclusive or Shared or Failover + // / Whether this subscription is Exclusive or Shared or Failover Type *string `protobuf:"bytes,13,opt,name=type" json:"type,omitempty"` - /// Total rate of messages expired on this subscription. msg/s + // / Total rate of messages expired on this subscription. msg/s MsgRateExpired *float64 `protobuf:"fixed64,14,opt,name=msgRateExpired" json:"msgRateExpired,omitempty"` - /// Number of messages in the subscription backlog + // / Number of messages in the subscription backlog MsgBacklog *uint64 `protobuf:"varint,15,opt,name=msgBacklog" json:"msgBacklog,omitempty"` - /// Total rate of messages ack. msg/s + // / Total rate of messages ack. msg/s MessageAckRate *float64 `protobuf:"fixed64,16,opt,name=messageAckRate" json:"messageAckRate,omitempty"` } @@ -8235,345 +8266,394 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, - 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, + 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, + 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, + 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, + 0x22, 0xd4, 0x01, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, 0xae, 0x01, 0x0a, 0x26, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, - 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x49, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, - 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x63, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xff, 0x01, 0x0a, 0x16, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, - 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, - 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x03, 0x3a, 0x02, 0x2d, 0x31, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, - 0x65, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, - 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, - 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2b, - 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0d, 0x70, 0x72, - 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x78, 0x0a, 0x0c, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x50, 0x69, 0x6e, 0x67, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, - 0x6f, 0x6e, 0x67, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x98, 0x05, 0x0a, 0x1c, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, - 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, - 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, - 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, - 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x01, 0x52, 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, - 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, - 0x52, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, - 0x65, 0x72, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, - 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, - 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, - 0x74, 0x73, 0x12, 0x28, 0x0a, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x75, 0x6e, 0x61, - 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1c, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, - 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x1c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x4f, 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, - 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, - 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, - 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, - 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x12, 0x1e, - 0x0a, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x18, 0x0f, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x12, 0x26, - 0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, - 0x18, 0x10, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, - 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, 0x22, 0x59, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, - 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x22, 0xe5, 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, - 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x1b, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x61, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x55, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x61, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x55, 0x72, 0x6c, 0x12, 0x40, 0x0a, 0x1b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, + 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x55, 0x72, + 0x6c, 0x54, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x61, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x65, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x55, 0x72, 0x6c, 0x54, 0x6c, 0x73, 0x22, 0xae, 0x01, 0x0a, 0x26, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, + 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, + 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, + 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, + 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x65, 0x70, + 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x22, 0x5d, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xff, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, + 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, + 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, + 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x3a, 0x02, 0x2d, 0x31, 0x52, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, + 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2b, 0x0a, 0x0e, + 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x78, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x02, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, + 0x6e, 0x67, 0x22, 0x0d, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, + 0x67, 0x22, 0x56, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x22, 0x98, 0x05, 0x0a, 0x1c, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, + 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x6d, 0x73, 0x67, 0x52, + 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x6d, 0x73, + 0x67, 0x52, 0x61, 0x74, 0x65, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x54, + 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, 0x74, 0x4f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x01, 0x52, 0x10, 0x6d, 0x73, 0x67, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, 0x68, 0x70, 0x75, + 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x10, + 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, + 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, + 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x10, + 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x74, 0x73, + 0x12, 0x28, 0x0a, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x75, 0x6e, 0x61, 0x63, 0x6b, + 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x42, 0x0a, 0x1c, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4f, 0x6e, 0x55, + 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x1c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x4f, 0x6e, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x18, + 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x53, 0x69, 0x6e, 0x63, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x6d, 0x73, 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, + 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x73, + 0x67, 0x52, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x12, 0x1e, 0x0a, 0x0a, + 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0a, 0x6d, 0x73, 0x67, 0x42, 0x61, 0x63, 0x6b, 0x6c, 0x6f, 0x67, 0x12, 0x26, 0x0a, 0x0e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, 0x52, 0x61, 0x74, 0x65, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x41, 0x63, 0x6b, + 0x52, 0x61, 0x74, 0x65, 0x22, 0x59, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, + 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, + 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, + 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x22, + 0xe5, 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, + 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x43, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x5e, 0x0a, 0x1d, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0d, 0x6c, 0x61, 0x73, - 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x5e, 0x0a, 0x1d, 0x63, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x1a, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x1b, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, - 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x44, 0x61, 0x74, 0x61, 0x52, 0x1a, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xa7, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x4d, 0x6f, + 0x64, 0x65, 0x3a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x52, 0x04, + 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, + 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x33, 0x0a, 0x04, + 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, + 0x4e, 0x54, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x4f, 0x4e, 0x5f, 0x50, 0x45, 0x52, 0x53, + 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, 0x4c, 0x10, + 0x02, 0x22, 0xc0, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, - 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, - 0x4d, 0x6f, 0x64, 0x65, 0x3a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, - 0x52, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, - 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x33, - 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x53, 0x49, 0x53, - 0x54, 0x45, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x4f, 0x4e, 0x5f, 0x50, 0x45, - 0x52, 0x53, 0x49, 0x53, 0x54, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4c, - 0x4c, 0x10, 0x02, 0x22, 0xc0, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, - 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, - 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x12, 0x21, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x08, 0x66, 0x69, 0x6c, - 0x74, 0x65, 0x72, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, - 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x12, 0x1e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, - 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x07, 0x63, - 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x22, 0xbb, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x12, 0x21, 0x0a, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x08, 0x3a, 0x05, 0x66, 0x61, 0x6c, 0x73, 0x65, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, + 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, + 0x48, 0x61, 0x73, 0x68, 0x12, 0x1e, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x08, 0x3a, 0x04, 0x74, 0x72, 0x75, 0x65, 0x52, 0x07, 0x63, 0x68, 0x61, + 0x6e, 0x67, 0x65, 0x64, 0x22, 0xbb, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1d, + 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, + 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, + 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, + 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, 0x09, 0x52, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x02, + 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, + 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, + 0x73, 0x68, 0x22, 0x93, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, + 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, + 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x9f, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x5a, 0x0a, 0x1a, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, + 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, + 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x22, 0x6e, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, + 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xed, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, + 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x7d, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, + 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, + 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x03, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc7, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, + 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, + 0x56, 0x0a, 0x1d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1c, - 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x02, 0x28, - 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x04, - 0x20, 0x02, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x50, 0x61, 0x74, 0x74, - 0x65, 0x72, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, - 0x73, 0x68, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, - 0x48, 0x61, 0x73, 0x68, 0x22, 0x93, 0x01, 0x0a, 0x1c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, - 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, - 0x72, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0a, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x9f, 0x01, 0x0a, 0x17, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, 0x63, 0x68, 0x65, - 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, 0x61, 0x74, 0x63, - 0x68, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x77, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x5f, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x73, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x02, 0x28, 0x09, - 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x48, 0x61, 0x73, 0x68, 0x22, 0x5a, 0x0a, 0x1a, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, + 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x77, 0x61, 0x74, - 0x63, 0x68, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x77, - 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x49, 0x64, 0x22, 0x6e, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xed, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, - 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, - 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x7d, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x02, - 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x02, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc7, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, - 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x0a, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x22, 0x56, 0x0a, 0x1d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x02, 0x28, 0x04, - 0x3a, 0x01, 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0x8a, 0x01, 0x0a, 0x1e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, + 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x54, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, + 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, + 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, + 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, + 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, + 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, + 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0xb1, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, + 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, + 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, + 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, + 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, + 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, - 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2f, 0x0a, 0x05, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, + 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, + 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, + 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, + 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, + 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x48, + 0x0a, 0x0c, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, + 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, + 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, + 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, + 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, + 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, + 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, + 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, + 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, + 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, + 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x0d, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, + 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, + 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, + 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, + 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, + 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, + 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, + 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, + 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x5f, 0x74, 0x74, - 0x6c, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x54, 0x74, 0x6c, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x16, 0x0a, 0x05, 0x74, 0x63, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, - 0x3a, 0x01, 0x30, 0x52, 0x04, 0x74, 0x63, 0x49, 0x64, 0x22, 0xd9, 0x01, 0x0a, 0x15, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa8, 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xb1, 0x01, 0x0a, 0x18, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, - 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, - 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, - 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, - 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, - 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, - 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, - 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, - 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, - 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, - 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, - 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x22, 0x48, 0x0a, 0x0c, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x02, 0x28, 0x09, 0x52, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x02, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x1b, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, - 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, - 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, - 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, - 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, - 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x0d, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x1d, 0x0a, - 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, - 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, - 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, - 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, - 0x42, 0x69, 0x74, 0x73, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd9, 0x01, 0x0a, - 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, + 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, + 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, + 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, + 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, + 0x6b, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, + 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, @@ -8586,27 +8666,30 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xa8, 0x02, 0x0a, 0x18, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, - 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, - 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, 0x73, 0x74, 0x5f, - 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0d, 0x74, - 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x09, 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, - 0x6e, 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, - 0x66, 0x5f, 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, - 0x74, 0x42, 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x22, 0xe4, 0x01, 0x0a, 0x20, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, - 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd5, 0x02, 0x0a, 0x1b, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, + 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, 0x6d, 0x6f, + 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, + 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, + 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x74, 0x78, + 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, 0x69, 0x64, + 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6c, + 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x42, 0x69, + 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, + 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, + 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, 0x69, 0x64, @@ -8619,489 +8702,452 @@ var file_PulsarApi_proto_rawDesc = []byte{ 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xd5, 0x02, 0x0a, 0x1b, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x3e, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x36, 0x0a, 0x0a, 0x74, 0x78, 0x6e, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, - 0x74, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x21, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x5f, 0x6f, 0x66, - 0x5f, 0x6c, 0x6f, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x1c, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, 0x73, 0x74, - 0x42, 0x69, 0x74, 0x73, 0x4f, 0x66, 0x4c, 0x6f, 0x77, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, - 0x72, 0x6b, 0x22, 0xe7, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, - 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x02, 0x28, 0x04, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x10, 0x74, 0x78, 0x6e, - 0x69, 0x64, 0x5f, 0x6c, 0x65, 0x61, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x3a, 0x01, 0x30, 0x52, 0x0e, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x42, 0x69, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x5f, - 0x6d, 0x6f, 0x73, 0x74, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x3a, - 0x01, 0x30, 0x52, 0x0d, 0x74, 0x78, 0x6e, 0x69, 0x64, 0x4d, 0x6f, 0x73, 0x74, 0x42, 0x69, 0x74, - 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9a, 0x2e, 0x0a, - 0x0b, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x32, 0x0a, 0x04, - 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, - 0x12, 0x36, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, - 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, + 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9a, 0x2e, 0x0a, 0x0b, 0x42, + 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x32, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x02, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x36, + 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x07, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x65, 0x64, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x12, 0x39, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x2d, 0x0a, + 0x04, 0x73, 0x65, 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x12, 0x3c, 0x0a, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x12, 0x43, 0x0a, 0x0c, + 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, + 0x65, 0x69, 0x70, 0x74, 0x52, 0x0b, 0x73, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70, + 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, + 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x12, 0x36, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, 0x6b, 0x52, + 0x03, 0x61, 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x46, 0x6c, 0x6f, 0x77, 0x52, 0x04, 0x66, + 0x6c, 0x6f, 0x77, 0x12, 0x42, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x55, + 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x0b, 0x75, 0x6e, 0x73, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, + 0x73, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x30, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x12, 0x49, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x09, 0x73, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x12, 0x39, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, - 0x2d, 0x0a, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x0d, 0x63, + 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x0e, + 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, + 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x43, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x72, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, + 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x69, 0x6e, 0x67, + 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x69, 0x6e, + 0x67, 0x52, 0x04, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x18, + 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, 0x6e, 0x67, + 0x52, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x12, 0x7e, 0x0a, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, + 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, + 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x34, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, + 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, 0x72, + 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x15, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x12, 0x73, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, + 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x04, 0x73, 0x65, 0x6e, 0x64, 0x12, 0x43, - 0x0a, 0x0c, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, 0x6e, 0x64, 0x52, - 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x52, 0x0b, 0x73, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x63, 0x65, - 0x69, 0x70, 0x74, 0x12, 0x3d, 0x0a, 0x0a, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, 0x65, - 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6e, 0x64, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x12, 0x36, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2a, 0x0a, 0x03, 0x61, 0x63, - 0x6b, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, - 0x6b, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x0b, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x46, 0x6c, 0x6f, 0x77, 0x52, - 0x04, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x42, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x0b, 0x75, 0x6e, - 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x73, 0x75, 0x63, - 0x63, 0x65, 0x73, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x07, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x12, 0x30, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x12, 0x49, 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x72, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, - 0x0d, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x12, 0x49, - 0x0a, 0x0e, 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6c, 0x6f, - 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x52, 0x0d, 0x63, 0x6c, 0x6f, 0x73, - 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x10, 0x70, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x65, 0x72, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x11, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, - 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, - 0x69, 0x6e, 0x67, 0x52, 0x04, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x04, 0x70, 0x6f, 0x6e, - 0x67, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x6f, - 0x6e, 0x67, 0x52, 0x04, 0x70, 0x6f, 0x6e, 0x67, 0x12, 0x7e, 0x0a, 0x1f, 0x72, 0x65, 0x64, 0x65, - 0x6c, 0x69, 0x76, 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, - 0x67, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x34, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, 0x65, - 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x1f, 0x72, 0x65, 0x64, 0x65, 0x6c, 0x69, 0x76, - 0x65, 0x72, 0x55, 0x6e, 0x61, 0x63, 0x6b, 0x6e, 0x6f, 0x77, 0x6c, 0x65, 0x64, 0x67, 0x65, 0x64, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x5b, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x15, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x73, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x6c, 0x6f, - 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x52, 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x5a, - 0x0a, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, - 0x74, 0x61, 0x74, 0x73, 0x12, 0x60, 0x0a, 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1a, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, - 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, - 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, - 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, - 0x65, 0x64, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2d, 0x0a, 0x04, - 0x73, 0x65, 0x65, 0x6b, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x12, 0x51, 0x0a, 0x10, 0x67, - 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x18, - 0x1d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, - 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x10, 0x67, 0x65, - 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x69, - 0x0a, 0x18, 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2d, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x18, 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, - 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x61, 0x63, 0x74, - 0x69, 0x76, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, - 0x6e, 0x67, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x6d, 0x61, 0x6e, 0x64, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, + 0x0b, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x5a, 0x0a, 0x13, + 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, - 0x61, 0x6e, 0x67, 0x65, 0x52, 0x14, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x67, 0x65, - 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, - 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x14, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, - 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x1c, 0x67, 0x65, 0x74, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x52, 0x13, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x12, 0x60, 0x0a, 0x15, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x15, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, + 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, 0x45, 0x6e, 0x64, + 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x11, 0x72, 0x65, 0x61, 0x63, 0x68, 0x65, 0x64, + 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x65, + 0x65, 0x6b, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x53, + 0x65, 0x65, 0x6b, 0x52, 0x04, 0x73, 0x65, 0x65, 0x6b, 0x12, 0x51, 0x0a, 0x10, 0x67, 0x65, 0x74, + 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x18, 0x1d, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, + 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x10, 0x67, 0x65, 0x74, 0x4c, + 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x69, 0x0a, 0x18, + 0x67, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, + 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x18, 0x67, + 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x16, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, + 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x63, + 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, + 0x67, 0x65, 0x52, 0x14, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x67, 0x65, 0x74, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, - 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, - 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, + 0x65, 0x52, 0x14, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, + 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x3c, 0x0a, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x22, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, - 0x0a, 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x52, 0x1c, 0x67, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x4f, 0x66, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3c, + 0x0a, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x22, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x09, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x54, 0x0a, 0x11, + 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, + 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, + 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, + 0x6e, 0x67, 0x65, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x52, 0x11, 0x67, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, - 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x52, - 0x0d, 0x61, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x12, 0x45, - 0x0a, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x25, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x41, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0b, 0x61, 0x63, - 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, - 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x27, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x67, 0x65, - 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, - 0x6c, 0x0a, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x28, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x41, 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x52, 0x0d, 0x61, + 0x75, 0x74, 0x68, 0x43, 0x68, 0x61, 0x6c, 0x6c, 0x65, 0x6e, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0c, + 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x25, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0c, 0x61, 0x75, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x0b, 0x61, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, + 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x67, 0x65, 0x74, 0x4f, 0x72, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x27, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, - 0x06, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x6e, 0x65, 0x77, 0x54, - 0x78, 0x6e, 0x12, 0x4b, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, - 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x54, 0x0a, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, - 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x52, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x67, 0x65, 0x74, 0x4f, + 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x6c, 0x0a, + 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x19, 0x67, 0x65, 0x74, 0x4f, 0x72, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x6e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, + 0x6e, 0x64, 0x4e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, + 0x12, 0x4b, 0x0a, 0x0e, 0x6e, 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x4e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x6e, + 0x65, 0x77, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, + 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, + 0x78, 0x6e, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x36, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x14, 0x61, 0x64, - 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, - 0x78, 0x6e, 0x12, 0x75, 0x0a, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, - 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, - 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, - 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x61, 0x64, 0x64, + 0x52, 0x11, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, + 0x54, 0x78, 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x61, 0x64, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x5d, 0x0a, 0x14, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x18, 0x36, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x14, 0x61, 0x64, 0x64, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, + 0x12, 0x75, 0x0a, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x65, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, - 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, - 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x4b, - 0x0a, 0x0e, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x18, 0x39, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x65, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x65, - 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x3a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, - 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, - 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x6c, 0x0a, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3b, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x61, 0x64, 0x64, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x54, 0x78, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, + 0x6e, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, + 0x64, 0x54, 0x78, 0x6e, 0x52, 0x06, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x12, 0x4b, 0x0a, 0x0e, + 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x39, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x5d, 0x0a, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, - 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x75, - 0x0a, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3d, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0e, 0x65, 0x6e, 0x64, 0x54, 0x78, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x11, 0x65, 0x6e, 0x64, + 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3a, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, - 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, - 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, - 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, - 0x3e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x52, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x66, 0x0a, 0x17, 0x74, 0x63, - 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x75, - 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x4c, 0x69, 0x73, 0x74, 0x18, 0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, - 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, - 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x52, - 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, - 0x60, 0x0a, 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, - 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x41, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, - 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, - 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, - 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x15, 0x77, 0x61, 0x74, 0x63, - 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x12, 0x51, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x18, 0x42, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, + 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x6e, + 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x6c, 0x0a, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3b, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x52, 0x19, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, + 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, - 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x52, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x43, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, - 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x52, 0x13, 0x77, 0x61, 0x74, - 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, - 0x22, 0x86, 0x0a, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x4e, - 0x4e, 0x45, 0x43, 0x54, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, - 0x54, 0x45, 0x44, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, - 0x42, 0x45, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, - 0x10, 0x05, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, - 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x43, 0x45, 0x49, 0x50, 0x54, 0x10, 0x07, 0x12, 0x0e, - 0x0a, 0x0a, 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x08, 0x12, 0x0b, - 0x0a, 0x07, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x41, - 0x43, 0x4b, 0x10, 0x0a, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x0b, 0x12, 0x0f, - 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x0c, 0x12, - 0x0b, 0x0a, 0x07, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x0d, 0x12, 0x09, 0x0a, 0x05, - 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x0e, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, - 0x5f, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x0f, 0x12, 0x12, 0x0a, 0x0e, 0x43, - 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x10, 0x12, - 0x14, 0x0a, 0x10, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x43, 0x43, - 0x45, 0x53, 0x53, 0x10, 0x11, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x49, 0x4e, 0x47, 0x10, 0x12, 0x12, - 0x08, 0x0a, 0x04, 0x50, 0x4f, 0x4e, 0x47, 0x10, 0x13, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x44, - 0x45, 0x4c, 0x49, 0x56, 0x45, 0x52, 0x5f, 0x55, 0x4e, 0x41, 0x43, 0x4b, 0x4e, 0x4f, 0x57, 0x4c, - 0x45, 0x44, 0x47, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x53, 0x10, 0x14, - 0x12, 0x18, 0x0a, 0x14, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, - 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x10, 0x15, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, - 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, - 0x54, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x16, 0x12, 0x0a, 0x0a, - 0x06, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x10, 0x17, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x4f, 0x4f, - 0x4b, 0x55, 0x50, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x18, 0x12, 0x12, - 0x0a, 0x0e, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, - 0x10, 0x19, 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, - 0x54, 0x41, 0x54, 0x53, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1a, 0x12, - 0x18, 0x0a, 0x14, 0x52, 0x45, 0x41, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x45, 0x4e, 0x44, 0x5f, 0x4f, - 0x46, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x10, 0x1b, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x45, - 0x4b, 0x10, 0x1c, 0x12, 0x17, 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, - 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1d, 0x12, 0x20, 0x0a, 0x1c, - 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, - 0x5f, 0x49, 0x44, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1e, 0x12, 0x1a, - 0x0a, 0x16, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, - 0x52, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1f, 0x12, 0x1b, 0x0a, 0x17, 0x47, 0x45, - 0x54, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, - 0x53, 0x50, 0x41, 0x43, 0x45, 0x10, 0x20, 0x12, 0x24, 0x0a, 0x20, 0x47, 0x45, 0x54, 0x5f, 0x54, - 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, - 0x43, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x21, 0x12, 0x0e, 0x0a, - 0x0a, 0x47, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x22, 0x12, 0x17, 0x0a, - 0x13, 0x47, 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, - 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x23, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x43, - 0x48, 0x41, 0x4c, 0x4c, 0x45, 0x4e, 0x47, 0x45, 0x10, 0x24, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x55, - 0x54, 0x48, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x25, 0x12, 0x10, 0x0a, - 0x0c, 0x41, 0x43, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x26, 0x12, - 0x18, 0x0a, 0x14, 0x47, 0x45, 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, - 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x27, 0x12, 0x21, 0x0a, 0x1d, 0x47, 0x45, 0x54, - 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, - 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x28, 0x12, 0x0b, 0x0a, 0x07, - 0x4e, 0x45, 0x57, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x32, 0x12, 0x14, 0x0a, 0x10, 0x4e, 0x45, 0x57, - 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x33, 0x12, - 0x18, 0x0a, 0x14, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x34, 0x12, 0x21, 0x0a, 0x1d, 0x41, 0x44, 0x44, - 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, - 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x35, 0x12, 0x1b, 0x0a, 0x17, - 0x41, 0x44, 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x36, 0x12, 0x24, 0x0a, 0x20, 0x41, 0x44, 0x44, - 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, - 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x37, 0x12, - 0x0b, 0x0a, 0x07, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x38, 0x12, 0x14, 0x0a, 0x10, - 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, - 0x10, 0x39, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, - 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3a, 0x12, 0x21, 0x0a, 0x1d, - 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, - 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3b, 0x12, - 0x1b, 0x0a, 0x17, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, - 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3c, 0x12, 0x24, 0x0a, 0x20, - 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, - 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, - 0x10, 0x3d, 0x12, 0x1d, 0x0a, 0x19, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, - 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x10, - 0x3e, 0x12, 0x1e, 0x0a, 0x1a, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, - 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, - 0x3f, 0x12, 0x14, 0x0a, 0x10, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, - 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x10, 0x40, 0x12, 0x1c, 0x0a, 0x18, 0x57, 0x41, 0x54, 0x43, 0x48, - 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x53, 0x55, 0x43, 0x43, - 0x45, 0x53, 0x53, 0x10, 0x41, 0x12, 0x16, 0x0a, 0x12, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, - 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x42, 0x12, 0x1a, 0x0a, - 0x16, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, - 0x54, 0x5f, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x10, 0x43, 0x2a, 0x44, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, - 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, - 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, 0x34, 0x10, 0x01, 0x12, - 0x08, 0x0a, 0x04, 0x5a, 0x4c, 0x49, 0x42, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x53, 0x54, - 0x44, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4e, 0x41, 0x50, 0x50, 0x59, 0x10, 0x04, 0x2a, - 0x5f, 0x0a, 0x12, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x10, - 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, 0x01, - 0x12, 0x14, 0x0a, 0x10, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x45, 0x78, 0x63, 0x6c, 0x75, - 0x73, 0x69, 0x76, 0x65, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, - 0x69, 0x76, 0x65, 0x57, 0x69, 0x74, 0x68, 0x46, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x10, 0x03, - 0x2a, 0x82, 0x05, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x12, 0x10, 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, - 0x65, 0x6e, 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x41, - 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, - 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x05, 0x12, 0x13, - 0x0a, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x52, 0x65, 0x61, 0x64, - 0x79, 0x10, 0x06, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, - 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, - 0x64, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x29, 0x0a, 0x25, 0x50, 0x72, - 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, - 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, - 0x6d, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x09, 0x12, 0x1b, 0x0a, 0x17, 0x55, 0x6e, 0x73, 0x75, - 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x72, - 0x72, 0x6f, 0x72, 0x10, 0x0a, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x6f, - 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0b, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, - 0x10, 0x0c, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x6f, - 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0d, 0x12, 0x13, 0x0a, 0x0f, 0x54, 0x6f, 0x6f, 0x4d, - 0x61, 0x6e, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x10, 0x0e, 0x12, 0x18, 0x0a, - 0x14, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, - 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x0f, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, - 0x61, 0x6c, 0x69, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x11, 0x12, - 0x16, 0x0a, 0x12, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x10, 0x12, 0x12, 0x17, 0x0a, 0x13, 0x43, 0x6f, 0x6e, 0x73, 0x75, - 0x6d, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x13, - 0x12, 0x22, 0x0a, 0x1e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, - 0x6e, 0x64, 0x10, 0x14, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, - 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x10, 0x15, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x6f, - 0x74, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x16, 0x12, - 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x6c, 0x69, 0x63, 0x74, 0x10, 0x17, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, - 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, - 0x18, 0x12, 0x12, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x46, 0x65, 0x6e, - 0x63, 0x65, 0x64, 0x10, 0x19, 0x2a, 0x4b, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, - 0x64, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x41, 0x75, 0x74, 0x68, 0x4d, - 0x65, 0x74, 0x68, 0x6f, 0x64, 0x59, 0x63, 0x61, 0x56, 0x31, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, - 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x74, 0x68, 0x65, 0x6e, 0x73, - 0x10, 0x02, 0x2a, 0xbb, 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x30, 0x10, 0x00, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x31, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x32, 0x10, 0x02, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x33, 0x10, 0x03, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x34, 0x10, 0x04, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x35, 0x10, 0x05, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x36, 0x10, 0x06, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x37, 0x10, 0x07, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x38, 0x10, 0x08, 0x12, 0x06, - 0x0a, 0x02, 0x76, 0x39, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x30, 0x10, 0x0a, 0x12, - 0x07, 0x0a, 0x03, 0x76, 0x31, 0x31, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x32, 0x10, - 0x0c, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x33, 0x10, 0x0d, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, - 0x34, 0x10, 0x0e, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x35, 0x10, 0x0f, 0x12, 0x07, 0x0a, 0x03, - 0x76, 0x31, 0x36, 0x10, 0x10, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x37, 0x10, 0x11, 0x12, 0x07, - 0x0a, 0x03, 0x76, 0x31, 0x38, 0x10, 0x12, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x39, 0x10, 0x13, - 0x2a, 0x2b, 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x4d, 0x6f, 0x64, - 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x55, 0x54, 0x4f, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x10, - 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x01, 0x2a, 0x22, 0x0a, - 0x09, 0x54, 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, - 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, - 0x01, 0x42, 0x2f, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, - 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x03, 0x5a, 0x07, 0x2e, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, + 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x75, 0x0a, 0x1c, + 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x3d, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x45, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x1c, 0x65, 0x6e, 0x64, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x3e, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x52, 0x16, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x66, 0x0a, 0x17, 0x74, 0x63, 0x43, 0x6c, + 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x18, 0x3f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x54, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x17, 0x74, 0x63, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x4b, 0x0a, 0x0e, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, + 0x73, 0x74, 0x18, 0x40, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, + 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, + 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x0e, 0x77, + 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x60, 0x0a, + 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x18, 0x41, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x70, + 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, + 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x52, 0x15, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x12, + 0x51, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x18, 0x42, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x75, 0x6c, 0x73, + 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x10, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x43, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x28, 0x2e, 0x70, 0x75, 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, + 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x57, 0x61, 0x74, 0x63, 0x68, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x52, 0x13, 0x77, 0x61, 0x74, 0x63, 0x68, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x86, + 0x0a, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x4f, 0x4e, 0x4e, 0x45, + 0x43, 0x54, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, + 0x44, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, + 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x05, + 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x4e, 0x44, 0x10, 0x06, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x45, + 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x43, 0x45, 0x49, 0x50, 0x54, 0x10, 0x07, 0x12, 0x0e, 0x0a, 0x0a, + 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x08, 0x12, 0x0b, 0x0a, 0x07, + 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x43, 0x4b, + 0x10, 0x0a, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x0b, 0x12, 0x0f, 0x0a, 0x0b, + 0x55, 0x4e, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x42, 0x45, 0x10, 0x0c, 0x12, 0x0b, 0x0a, + 0x07, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, 0x53, 0x10, 0x0d, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, + 0x52, 0x4f, 0x52, 0x10, 0x0e, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x5f, 0x50, + 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x10, 0x0f, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4c, 0x4f, + 0x53, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x10, 0x10, 0x12, 0x14, 0x0a, + 0x10, 0x50, 0x52, 0x4f, 0x44, 0x55, 0x43, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, + 0x53, 0x10, 0x11, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x49, 0x4e, 0x47, 0x10, 0x12, 0x12, 0x08, 0x0a, + 0x04, 0x50, 0x4f, 0x4e, 0x47, 0x10, 0x13, 0x12, 0x25, 0x0a, 0x21, 0x52, 0x45, 0x44, 0x45, 0x4c, + 0x49, 0x56, 0x45, 0x52, 0x5f, 0x55, 0x4e, 0x41, 0x43, 0x4b, 0x4e, 0x4f, 0x57, 0x4c, 0x45, 0x44, + 0x47, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x53, 0x10, 0x14, 0x12, 0x18, + 0x0a, 0x14, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, + 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x10, 0x15, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x41, 0x52, 0x54, + 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x45, 0x44, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, + 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x16, 0x12, 0x0a, 0x0a, 0x06, 0x4c, + 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x10, 0x17, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, + 0x50, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x18, 0x12, 0x12, 0x0a, 0x0e, + 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x53, 0x10, 0x19, + 0x12, 0x1b, 0x0a, 0x17, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x53, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1a, 0x12, 0x18, 0x0a, + 0x14, 0x52, 0x45, 0x41, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x45, 0x4e, 0x44, 0x5f, 0x4f, 0x46, 0x5f, + 0x54, 0x4f, 0x50, 0x49, 0x43, 0x10, 0x1b, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x45, 0x45, 0x4b, 0x10, + 0x1c, 0x12, 0x17, 0x0a, 0x13, 0x47, 0x45, 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, + 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1d, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x45, + 0x54, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, + 0x44, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x1e, 0x12, 0x1a, 0x0a, 0x16, + 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x52, 0x5f, + 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1f, 0x12, 0x1b, 0x0a, 0x17, 0x47, 0x45, 0x54, 0x5f, + 0x54, 0x4f, 0x50, 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, + 0x41, 0x43, 0x45, 0x10, 0x20, 0x12, 0x24, 0x0a, 0x20, 0x47, 0x45, 0x54, 0x5f, 0x54, 0x4f, 0x50, + 0x49, 0x43, 0x53, 0x5f, 0x4f, 0x46, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, + 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x21, 0x12, 0x0e, 0x0a, 0x0a, 0x47, + 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x22, 0x12, 0x17, 0x0a, 0x13, 0x47, + 0x45, 0x54, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, + 0x53, 0x45, 0x10, 0x23, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x55, 0x54, 0x48, 0x5f, 0x43, 0x48, 0x41, + 0x4c, 0x4c, 0x45, 0x4e, 0x47, 0x45, 0x10, 0x24, 0x12, 0x11, 0x0a, 0x0d, 0x41, 0x55, 0x54, 0x48, + 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x25, 0x12, 0x10, 0x0a, 0x0c, 0x41, + 0x43, 0x4b, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x26, 0x12, 0x18, 0x0a, + 0x14, 0x47, 0x45, 0x54, 0x5f, 0x4f, 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, + 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x27, 0x12, 0x21, 0x0a, 0x1d, 0x47, 0x45, 0x54, 0x5f, 0x4f, + 0x52, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x28, 0x12, 0x0b, 0x0a, 0x07, 0x4e, 0x45, + 0x57, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x32, 0x12, 0x14, 0x0a, 0x10, 0x4e, 0x45, 0x57, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x33, 0x12, 0x18, 0x0a, + 0x14, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x34, 0x12, 0x21, 0x0a, 0x1d, 0x41, 0x44, 0x44, 0x5f, 0x50, + 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x5f, + 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x35, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x44, + 0x44, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x4f, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x36, 0x12, 0x24, 0x0a, 0x20, 0x41, 0x44, 0x44, 0x5f, 0x53, + 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x4f, 0x5f, 0x54, + 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x37, 0x12, 0x0b, 0x0a, + 0x07, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x10, 0x38, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x39, + 0x12, 0x18, 0x0a, 0x14, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, + 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3a, 0x12, 0x21, 0x0a, 0x1d, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, + 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3b, 0x12, 0x1b, 0x0a, + 0x17, 0x45, 0x4e, 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, + 0x43, 0x52, 0x49, 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x3c, 0x12, 0x24, 0x0a, 0x20, 0x45, 0x4e, + 0x44, 0x5f, 0x54, 0x58, 0x4e, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x55, 0x42, 0x53, 0x43, 0x52, 0x49, + 0x50, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3d, + 0x12, 0x1d, 0x0a, 0x19, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, + 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x10, 0x3e, 0x12, + 0x1e, 0x0a, 0x1a, 0x54, 0x43, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x4e, + 0x4e, 0x45, 0x43, 0x54, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x10, 0x3f, 0x12, + 0x14, 0x0a, 0x10, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, + 0x49, 0x53, 0x54, 0x10, 0x40, 0x12, 0x1c, 0x0a, 0x18, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, + 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, 0x53, 0x55, 0x43, 0x43, 0x45, 0x53, + 0x53, 0x10, 0x41, 0x12, 0x16, 0x0a, 0x12, 0x57, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, + 0x49, 0x43, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x42, 0x12, 0x1a, 0x0a, 0x16, 0x57, + 0x41, 0x54, 0x43, 0x48, 0x5f, 0x54, 0x4f, 0x50, 0x49, 0x43, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x5f, + 0x43, 0x4c, 0x4f, 0x53, 0x45, 0x10, 0x43, 0x2a, 0x44, 0x0a, 0x0f, 0x43, 0x6f, 0x6d, 0x70, 0x72, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, + 0x4e, 0x45, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, 0x34, 0x10, 0x01, 0x12, 0x08, 0x0a, + 0x04, 0x5a, 0x4c, 0x49, 0x42, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x5a, 0x53, 0x54, 0x44, 0x10, + 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x4e, 0x41, 0x50, 0x50, 0x59, 0x10, 0x04, 0x2a, 0x5f, 0x0a, + 0x12, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4d, + 0x6f, 0x64, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x10, 0x00, 0x12, + 0x0d, 0x0a, 0x09, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x10, 0x01, 0x12, 0x14, + 0x0a, 0x10, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, + 0x76, 0x65, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, + 0x65, 0x57, 0x69, 0x74, 0x68, 0x46, 0x65, 0x6e, 0x63, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x2a, 0x82, + 0x05, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x10, + 0x0a, 0x0c, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x00, + 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x6e, + 0x63, 0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x02, 0x12, 0x17, 0x0a, 0x13, 0x41, 0x75, 0x74, + 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x7a, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x74, 0x52, 0x65, 0x61, 0x64, 0x79, 0x10, + 0x06, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, + 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x07, 0x12, 0x29, 0x0a, 0x25, 0x50, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x72, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x65, 0x64, 0x51, 0x75, 0x6f, 0x74, 0x61, + 0x45, 0x78, 0x63, 0x65, 0x65, 0x64, 0x65, 0x64, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x10, 0x09, 0x12, 0x1b, 0x0a, 0x17, 0x55, 0x6e, 0x73, 0x75, 0x70, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, + 0x72, 0x10, 0x0a, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x6f, 0x74, 0x46, + 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0b, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0c, + 0x12, 0x14, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x4e, 0x6f, 0x74, 0x46, + 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x0d, 0x12, 0x13, 0x0a, 0x0f, 0x54, 0x6f, 0x6f, 0x4d, 0x61, 0x6e, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x10, 0x0e, 0x12, 0x18, 0x0a, 0x14, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x10, 0x0f, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, + 0x72, 0x42, 0x75, 0x73, 0x79, 0x10, 0x10, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, + 0x69, 0x64, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x11, 0x12, 0x16, 0x0a, + 0x12, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x10, 0x12, 0x12, 0x17, 0x0a, 0x13, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x13, 0x12, 0x22, + 0x0a, 0x1e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6f, + 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, + 0x10, 0x14, 0x12, 0x14, 0x0a, 0x10, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x54, 0x78, 0x6e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x10, 0x15, 0x12, 0x13, 0x0a, 0x0f, 0x4e, 0x6f, 0x74, 0x41, + 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x16, 0x12, 0x17, 0x0a, + 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x6c, 0x69, 0x63, 0x74, 0x10, 0x17, 0x12, 0x17, 0x0a, 0x13, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64, 0x10, 0x18, 0x12, + 0x12, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x46, 0x65, 0x6e, 0x63, 0x65, + 0x64, 0x10, 0x19, 0x2a, 0x4b, 0x0a, 0x0a, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4e, + 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x65, 0x74, + 0x68, 0x6f, 0x64, 0x59, 0x63, 0x61, 0x56, 0x31, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x41, 0x75, + 0x74, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x74, 0x68, 0x65, 0x6e, 0x73, 0x10, 0x02, + 0x2a, 0xbb, 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x30, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x31, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x32, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x33, 0x10, 0x03, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x34, 0x10, 0x04, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x35, 0x10, 0x05, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x36, 0x10, 0x06, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x37, 0x10, 0x07, 0x12, 0x06, 0x0a, 0x02, 0x76, 0x38, 0x10, 0x08, 0x12, 0x06, 0x0a, 0x02, + 0x76, 0x39, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x30, 0x10, 0x0a, 0x12, 0x07, 0x0a, + 0x03, 0x76, 0x31, 0x31, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x32, 0x10, 0x0c, 0x12, + 0x07, 0x0a, 0x03, 0x76, 0x31, 0x33, 0x10, 0x0d, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x34, 0x10, + 0x0e, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x35, 0x10, 0x0f, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, + 0x36, 0x10, 0x10, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x37, 0x10, 0x11, 0x12, 0x07, 0x0a, 0x03, + 0x76, 0x31, 0x38, 0x10, 0x12, 0x12, 0x07, 0x0a, 0x03, 0x76, 0x31, 0x39, 0x10, 0x13, 0x2a, 0x2b, + 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x53, 0x68, 0x61, 0x72, 0x65, 0x64, 0x4d, 0x6f, 0x64, 0x65, 0x12, + 0x0e, 0x0a, 0x0a, 0x41, 0x55, 0x54, 0x4f, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x10, 0x00, 0x12, + 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x49, 0x43, 0x4b, 0x59, 0x10, 0x01, 0x2a, 0x22, 0x0a, 0x09, 0x54, + 0x78, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, + 0x49, 0x54, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x42, 0x4f, 0x52, 0x54, 0x10, 0x01, 0x42, + 0x2f, 0x0a, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x70, 0x75, + 0x6c, 0x73, 0x61, 0x72, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x03, 0x5a, 0x07, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, } var ( diff --git a/pulsar/internal/pulsar_proto/PulsarApi.proto b/pulsar/internal/pulsar_proto/PulsarApi.proto index 8bd75729c8..a95a114802 100644 --- a/pulsar/internal/pulsar_proto/PulsarApi.proto +++ b/pulsar/internal/pulsar_proto/PulsarApi.proto @@ -625,11 +625,15 @@ message CommandReachedEndOfTopic { message CommandCloseProducer { required uint64 producer_id = 1; required uint64 request_id = 2; + optional string assignedBrokerServiceUrl = 3; + optional string assignedBrokerServiceUrlTls = 4; } message CommandCloseConsumer { required uint64 consumer_id = 1; required uint64 request_id = 2; + optional string assignedBrokerServiceUrl = 3; + optional string assignedBrokerServiceUrlTls = 4; } message CommandRedeliverUnacknowledgedMessages { diff --git a/pulsar/producer_partition.go b/pulsar/producer_partition.go index fbcc5b9776..f5c838c4d3 100755 --- a/pulsar/producer_partition.go +++ b/pulsar/producer_partition.go @@ -107,7 +107,7 @@ type partitionProducer struct { // Channel where app is posting messages to be published dataChan chan *sendRequest cmdChan chan interface{} - connectClosedCh chan connectionClosed + connectClosedCh chan *connectionClosed publishSemaphore internal.Semaphore pendingQueue internal.BlockingQueue @@ -168,7 +168,7 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions producerID: client.rpcClient.NewProducerID(), dataChan: make(chan *sendRequest, maxPendingMessages), cmdChan: make(chan interface{}, 10), - connectClosedCh: make(chan connectionClosed, 10), + connectClosedCh: make(chan *connectionClosed, 10), batchFlushTicker: time.NewTicker(batchingMaxPublishDelay), compressionProvider: internal.GetCompressionProvider(pb.CompressionType(options.CompressionType), compression.Level(options.CompressionLevel)), @@ -197,7 +197,7 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions } else { p.userProvidedProducerName = false } - err := p.grabCnx() + err := p.grabCnx("") if err != nil { p.batchFlushTicker.Stop() logger.WithError(err).Error("Failed to create producer at newPartitionProducer") @@ -221,14 +221,25 @@ func newPartitionProducer(client *client, topic string, options *ProducerOptions return p, nil } -func (p *partitionProducer) grabCnx() error { - lr, err := p.client.lookupService.Lookup(p.topic) +func (p *partitionProducer) lookupTopic(brokerServiceURL string) (*internal.LookupResult, error) { + if len(brokerServiceURL) == 0 { + lr, err := p.client.lookupService.Lookup(p.topic) + if err != nil { + p.log.WithError(err).Warn("Failed to lookup topic") + return nil, err + } + + p.log.Debug("Lookup result: ", lr) + return lr, err + } + return p.client.lookupService.GetBrokerAddress(brokerServiceURL, p._getConn().IsProxied()) +} + +func (p *partitionProducer) grabCnx(assignedBrokerURL string) error { + lr, err := p.lookupTopic(assignedBrokerURL) if err != nil { - p.log.WithError(err).Warn("Failed to lookup topic") return err } - - p.log.Debug("Lookup result: ", lr) id := p.client.rpcClient.NewRequestID() // set schema info for producer @@ -363,7 +374,13 @@ func (p *partitionProducer) grabCnx() error { return nil } -type connectionClosed struct{} +type connectionClosed struct { + assignedBrokerURL string +} + +func (cc *connectionClosed) HasURL() bool { + return len(cc.assignedBrokerURL) > 0 +} func (p *partitionProducer) GetBuffer() internal.Buffer { b, ok := buffersPool.Get().(internal.Buffer) @@ -373,10 +390,17 @@ func (p *partitionProducer) GetBuffer() internal.Buffer { return b } -func (p *partitionProducer) ConnectionClosed() { +func (p *partitionProducer) ConnectionClosed(closeProducer *pb.CommandCloseProducer) { // Trigger reconnection in the produce goroutine p.log.WithField("cnx", p._getConn().ID()).Warn("Connection was closed") - p.connectClosedCh <- connectionClosed{} + var assignedBrokerURL string + if closeProducer != nil { + assignedBrokerURL = p.client.selectServiceURL( + closeProducer.GetAssignedBrokerServiceUrl(), closeProducer.GetAssignedBrokerServiceUrlTls()) + } + p.connectClosedCh <- &connectionClosed{ + assignedBrokerURL: assignedBrokerURL, + } } func (p *partitionProducer) getOrCreateSchema(schemaInfo *SchemaInfo) (schemaVersion []byte, err error) { @@ -409,7 +433,7 @@ func (p *partitionProducer) getOrCreateSchema(schemaInfo *SchemaInfo) (schemaVer return res.Response.GetOrCreateSchemaResponse.SchemaVersion, nil } -func (p *partitionProducer) reconnectToBroker() { +func (p *partitionProducer) reconnectToBroker(connectionClosed *connectionClosed) { var maxRetry int if p.options.MaxReconnectToBroker == nil { maxRetry = -1 @@ -429,12 +453,22 @@ func (p *partitionProducer) reconnectToBroker() { return } - if p.options.BackoffPolicy == nil { + var assignedBrokerURL string + + if connectionClosed != nil && connectionClosed.HasURL() { + delayReconnectTime = 0 + assignedBrokerURL = connectionClosed.assignedBrokerURL + connectionClosed = nil // Only attempt once + } else if p.options.BackoffPolicy == nil { delayReconnectTime = defaultBackoff.Next() } else { delayReconnectTime = p.options.BackoffPolicy.Next() } - p.log.Info("Reconnecting to broker in ", delayReconnectTime) + + p.log.WithFields(log.Fields{ + "assignedBrokerURL": assignedBrokerURL, + "delayReconnectTime": delayReconnectTime, + }).Info("Reconnecting to broker") time.Sleep(delayReconnectTime) // double check @@ -445,7 +479,7 @@ func (p *partitionProducer) reconnectToBroker() { } atomic.AddUint64(&p.epoch, 1) - err := p.grabCnx() + err := p.grabCnx(assignedBrokerURL) if err == nil { // Successfully reconnected p.log.WithField("cnx", p._getConn().ID()).Info("Reconnected producer to broker") @@ -509,9 +543,9 @@ func (p *partitionProducer) runEventsLoop() { p.internalClose(v) return } - case <-p.connectClosedCh: + case connectionClosed := <-p.connectClosedCh: p.log.Info("runEventsLoop will reconnect in producer") - p.reconnectToBroker() + p.reconnectToBroker(connectionClosed) case <-p.batchFlushTicker.C: p.internalFlushCurrentBatch() } diff --git a/pulsar/producer_test.go b/pulsar/producer_test.go index ba5911565e..7c4ff89752 100644 --- a/pulsar/producer_test.go +++ b/pulsar/producer_test.go @@ -1285,22 +1285,22 @@ func TestProducerWithBackoffPolicy(t *testing.T) { partitionProducerImp := _producer.(*producer).producers[0].(*partitionProducer) // 1 s startTime := time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 2 s startTime = time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionProducerImp.reconnectToBroker() + partitionProducerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } diff --git a/pulsar/reader_test.go b/pulsar/reader_test.go index 93787d106c..a9c45ba806 100644 --- a/pulsar/reader_test.go +++ b/pulsar/reader_test.go @@ -878,22 +878,22 @@ func TestReaderWithBackoffPolicy(t *testing.T) { partitionConsumerImp := _reader.(*reader).c.consumers[0] // 1 s startTime := time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 2 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) // 4 s startTime = time.Now() - partitionConsumerImp.reconnectToBroker() + partitionConsumerImp.reconnectToBroker(nil) assert.True(t, backoff.IsExpectedIntervalFrom(startTime)) } diff --git a/scripts/run-ci-extensible-load-manager.sh b/scripts/run-ci-extensible-load-manager.sh new file mode 100755 index 0000000000..2fe5f7c229 --- /dev/null +++ b/scripts/run-ci-extensible-load-manager.sh @@ -0,0 +1,23 @@ +#!/bin/bash +# 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. + +set -e -x + +go test -race -coverprofile=/tmp/coverage -timeout=5m -tags extensible_load_manager -v -run TestExtensibleLoadManagerTestSuite ./pulsar +go tool cover -html=/tmp/coverage -o coverage.html +