[improve] PIP-307: Use assigned broker URL hints during broker reconnection (#1208)
diff --git a/Makefile b/Makefile
index cdae8a5..4e119fc 100644
--- a/Makefile
+++ b/Makefile
@@ -50,17 +50,23 @@
--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 0000000..935afdc
--- /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 65aed3b..8471e1e 100644
--- a/pulsar/client_impl.go
+++ b/pulsar/client_impl.go
@@ -49,6 +49,7 @@
memLimit internal.MemoryLimitController
closeOnce sync.Once
operationTimeout time.Duration
+ tlsEnabled bool
log log.Logger
}
@@ -166,6 +167,7 @@
metrics: metrics,
memLimit: internal.NewMemoryLimitController(memLimitBytes, defaultMemoryLimitTriggerThreshold),
operationTimeout: operationTimeout,
+ tlsEnabled: tlsConfig != nil,
}
serviceNameResolver := internal.NewPulsarServiceNameResolver(url)
@@ -174,7 +176,7 @@
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 @@
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 @@
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 162565b..dc01e69 100644
--- a/pulsar/consumer_partition.go
+++ b/pulsar/consumer_partition.go
@@ -158,7 +158,7 @@
eventsCh chan interface{}
connectedCh chan struct{}
- connectClosedCh chan connectionClosed
+ connectClosedCh chan *connectionClosed
closeCh chan struct{}
clearQueueCh chan func(id *trackingMessageID)
@@ -326,7 +326,7 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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) 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 4a3b532..4120ba4 100644
--- a/pulsar/consumer_test.go
+++ b/pulsar/consumer_test.go
@@ -3796,22 +3796,22 @@
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 0000000..b3025c3
--- /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 cb8c680..9244328 100644
--- a/pulsar/internal/connection.go
+++ b/pulsar/internal/connection.go
@@ -69,7 +69,7 @@
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 @@
ID() string
GetMaxMessageSize() int32
Close()
+ IsProxied() bool
}
type ConsumerHandler interface {
@@ -92,7 +93,7 @@
ActiveConsumerChanged(isActive bool)
// ConnectionClosed close the TCP connection.
- ConnectionClosed()
+ ConnectionClosed(closeConsumer *pb.CommandCloseConsumer)
}
type connectionState int32
@@ -312,7 +313,7 @@
},
}
- 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 @@
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 @@
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 @@
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 @@
// 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 e30bec1..8b7fb01 100644
--- a/pulsar/internal/lookup_service.go
+++ b/pulsar/internal/lookup_service.go
@@ -64,6 +64,8 @@
// 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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 c8e6ad9..5c6e0f3 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 @@
// 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 @@
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 @@
// 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 @@
// 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 @@
// 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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
return nil
}
-/// Response from CommandProducer
+// / Response from CommandProducer
type CommandProducerSuccess struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -4925,31 +4956,31 @@
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 @@
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,
- 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,
+ 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, 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, 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, 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, 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,
+ 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, 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, 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, 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,
+ 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 @@
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 @@
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,
+ 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, 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, 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,
+ 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,
+ 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, 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, 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,
+ 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,
- 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, 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,
- 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,
+ 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, 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,
+ 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, 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,
+ 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, 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, 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, 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, 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,
+ 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 8bd7572..a95a114 100644
--- a/pulsar/internal/pulsar_proto/PulsarApi.proto
+++ b/pulsar/internal/pulsar_proto/PulsarApi.proto
@@ -625,11 +625,15 @@
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 fbcc5b9..f5c838c 100755
--- a/pulsar/producer_partition.go
+++ b/pulsar/producer_partition.go
@@ -107,7 +107,7 @@
// 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 @@
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 @@
} 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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
}
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 @@
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 ba59115..7c4ff89 100644
--- a/pulsar/producer_test.go
+++ b/pulsar/producer_test.go
@@ -1285,22 +1285,22 @@
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 93787d1..a9c45ba 100644
--- a/pulsar/reader_test.go
+++ b/pulsar/reader_test.go
@@ -878,22 +878,22 @@
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 0000000..2fe5f7c
--- /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
+