From e0e600c7d877529e9fc61d29088851688b0c53e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Wed, 15 Mar 2023 13:12:06 +0100 Subject: [PATCH] Parallel processors (#744) * spike out parallel processor * simplify by spawning workers in advance * ParallelNode * add test case for a single failing worker * refactor parallel node * add extensive tests for parallel worker and coordinator * cleanup and docs * use pool for done channels * validate processor workers in pipeline config file * add processor workers validation in service * use temp folder in runtime test * use is instead of assert in runtime test * clarify Message.StatusError usage in parallel node * move comment * refactor + comments --- pkg/conduit/runtime_test.go | 21 +- pkg/foundation/log/fields.go | 19 +- pkg/foundation/semaphore/semaphore.go | 10 +- pkg/pipeline/lifecycle.go | 41 +- pkg/pipeline/stream/base.go | 8 +- pkg/pipeline/stream/logger.go | 33 +- pkg/pipeline/stream/message.go | 17 +- pkg/pipeline/stream/parallel.go | 367 +++++ pkg/pipeline/stream/parallel_test.go | 617 ++++++++ pkg/processor/instance.go | 1 + pkg/processor/service.go | 7 + pkg/processor/service_test.go | 130 +- pkg/provisioning/enrich.go | 3 + pkg/provisioning/enrich_test.go | 9 +- pkg/provisioning/parse.go | 1 + pkg/provisioning/service.go | 1 + pkg/provisioning/service_test.go | 51 +- .../provision1-pipeline-with-processors.yml | 1 + pkg/provisioning/validate.go | 5 +- pkg/provisioning/validate_test.go | 17 +- pkg/web/api/fromproto/processor.go | 1 + pkg/web/api/toproto/processor.go | 1 + proto/api/v1/api.proto | 1 + proto/gen/api/v1/api.pb.go | 1410 +++++++++-------- proto/gen/api/v1/api.swagger.json | 4 + 25 files changed, 1962 insertions(+), 814 deletions(-) create mode 100644 pkg/pipeline/stream/parallel.go create mode 100644 pkg/pipeline/stream/parallel_test.go diff --git a/pkg/conduit/runtime_test.go b/pkg/conduit/runtime_test.go index 6549cd67f..284d9ba86 100644 --- a/pkg/conduit/runtime_test.go +++ b/pkg/conduit/runtime_test.go @@ -16,35 +16,32 @@ package conduit_test import ( "context" - "os" "testing" "time" "github.com/conduitio/conduit/pkg/conduit" - "github.com/conduitio/conduit/pkg/foundation/assert" "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/matryer/is" ) // path where tests store their data during runs. -const testingDBPath = "./testing.app.db" const delay = 500 * time.Millisecond func TestRuntime(t *testing.T) { + is := is.New(t) + var cfg conduit.Config cfg.DB.Type = "badger" - cfg.DB.Badger.Path = testingDBPath + cfg.DB.Badger.Path = t.TempDir() + "/testing.app.db" cfg.GRPC.Address = ":0" cfg.HTTP.Address = ":0" cfg.Log.Level = "info" cfg.Log.Format = "cli" cfg.Pipelines.Path = "./pipelines" - e, err := conduit.NewRuntime(cfg) - t.Cleanup(func() { - os.RemoveAll(testingDBPath) - }) - assert.Ok(t, err) - assert.NotNil(t, e) + r, err := conduit.NewRuntime(cfg) + is.NoErr(err) + is.True(r != nil) // set a cancel on a trigger to kill the context after THRESHOLD duration. ctx, cancel := context.WithCancel(context.TODO()) @@ -55,6 +52,6 @@ func TestRuntime(t *testing.T) { // wait on Run and assert that the context was canceled and no other error // occurred. - err = e.Run(ctx) - assert.True(t, cerrors.Is(err, context.Canceled), "expected error to be context.Cancelled") + err = r.Run(ctx) + is.True(cerrors.Is(err, context.Canceled)) // expected error to be context.Cancelled } diff --git a/pkg/foundation/log/fields.go b/pkg/foundation/log/fields.go index e70c7fc96..3c4699546 100644 --- a/pkg/foundation/log/fields.go +++ b/pkg/foundation/log/fields.go @@ -15,15 +15,16 @@ package log const ( - ComponentField = "component" - ConnectorIDField = "connector_id" - DurationField = "duration" - MessageIDField = "message_id" - NodeIDField = "node_id" - PipelineIDField = "pipeline_id" - RecordPositionField = "record_position" - RequestIDField = "request_id" - ServerAddressField = "address" + ComponentField = "component" + ConnectorIDField = "connector_id" + DurationField = "duration" + MessageIDField = "message_id" + NodeIDField = "node_id" + ParallelWorkerIDField = "parallel_worker_id" + PipelineIDField = "pipeline_id" + RecordPositionField = "record_position" + RequestIDField = "request_id" + ServerAddressField = "address" GRPCMethodField = "grpc_method" GRPCStatusCodeField = "grpc_status_code" diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index 465998baa..e8cacdba7 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -31,7 +31,7 @@ type Simple struct { // Ticket reserves a place in the queue and can be used to acquire access to a // Lock. type Ticket struct { - Lock + l Lock // ready is the same channel as Lock.next of the previous lock. Once the // previous lock is released, the value i will be sent into ready, signaling // this ticket that it can acquire a Lock. @@ -64,13 +64,13 @@ func (s *Simple) Enqueue() Ticket { }) t := Ticket{ - Lock: Lock{ + l: Lock{ i: s.last.i + 1, next: s.chanPool.Get().(chan int), }, ready: s.last.next, } - s.last = t.Lock + s.last = t.l return t } @@ -80,7 +80,7 @@ func (s *Simple) Enqueue() Ticket { // Acquire is safe for concurrent use. func (s *Simple) Acquire(t Ticket) Lock { i := <-t.ready - if t.i != i { + if t.l.i != i { // Multiple reasons this can happen. A ticket other than this one could // have been successfully released twice. The other possibility is that // this ticket is trying to be acquired after it was released. Both @@ -88,7 +88,7 @@ func (s *Simple) Acquire(t Ticket) Lock { s.panic() } s.chanPool.Put(t.ready) - return t.Lock // return only the lock part of the ticket + return t.l // return only the lock part of the ticket } // Release releases the lock and allows the next ticket in line to acquire a diff --git a/pkg/pipeline/lifecycle.go b/pkg/pipeline/lifecycle.go index 7dd3cc8c1..a112cb6e5 100644 --- a/pkg/pipeline/lifecycle.go +++ b/pkg/pipeline/lifecycle.go @@ -17,6 +17,7 @@ package pipeline import ( "context" "fmt" + "strconv" "strings" "sync" "sync/atomic" @@ -288,21 +289,49 @@ func (s *Service) buildProcessorNodes( return nil, cerrors.Errorf("could not fetch processor: %w", err) } - node := stream.ProcessorNode{ - Name: proc.ID, - Processor: proc.Processor, - ProcessorTimer: measure.ProcessorExecutionDurationTimer.WithValues(pl.Config.Name, proc.Type), + var node stream.PubSubNode + if proc.Config.Workers > 1 { + node = s.buildParallelProcessorNode(pl, proc) + } else { + node = s.buildProcessorNode(pl, proc) } + node.Sub(prev.Pub()) - prev = &node + prev = node - nodes = append(nodes, &node) + nodes = append(nodes, node) } last.Sub(prev.Pub()) return nodes, nil } +func (s *Service) buildParallelProcessorNode( + pl *Instance, + proc *processor.Instance, +) *stream.ParallelNode { + return &stream.ParallelNode{ + Name: proc.ID + "-parallel", + NewNode: func(i int) stream.PubSubNode { + n := s.buildProcessorNode(pl, proc) + n.Name = n.Name + "-" + strconv.Itoa(i) // add suffix to name + return n + }, + Workers: proc.Config.Workers, + } +} + +func (s *Service) buildProcessorNode( + pl *Instance, + proc *processor.Instance, +) *stream.ProcessorNode { + return &stream.ProcessorNode{ + Name: proc.ID, + Processor: proc.Processor, + ProcessorTimer: measure.ProcessorExecutionDurationTimer.WithValues(pl.Config.Name, proc.Type), + } +} + func (s *Service) buildSourceNodes( ctx context.Context, connFetcher ConnectorFetcher, diff --git a/pkg/pipeline/stream/base.go b/pkg/pipeline/stream/base.go index 906b1b60f..1171c8549 100644 --- a/pkg/pipeline/stream/base.go +++ b/pkg/pipeline/stream/base.go @@ -348,13 +348,17 @@ func (n *nodeBase) Send( if msg.Ctx == nil { msg.Ctx = ctxutil.ContextWithMessageID(ctx, msg.ID()) } + // copy context into a local variable, we shouldn't access it anymore after + // we send the message to out, this prevents race conditions in case the + // field gets changed + msgCtx := msg.Ctx select { case <-ctx.Done(): - logger.Debug(msg.Ctx).Msg("context closed while sending message") + logger.Debug(msgCtx).Msg("context closed while sending message") return ctx.Err() case out <- msg: - logger.Trace(msg.Ctx).Msg("sent message to outgoing channel") + logger.Trace(msgCtx).Msg("sent message to outgoing channel") } return nil } diff --git a/pkg/pipeline/stream/logger.go b/pkg/pipeline/stream/logger.go index 625a7ba4b..75fe2bf77 100644 --- a/pkg/pipeline/stream/logger.go +++ b/pkg/pipeline/stream/logger.go @@ -21,20 +21,39 @@ import ( ) // SetLogger figures out if the node needs a logger, sets static metadata in the -// logger and supplies it to the node. -func SetLogger(n Node, logger log.CtxLogger) { +// logger and supplies it to the node. Behavior can be customized by supplying +// custom options. +func SetLogger(n Node, logger log.CtxLogger, options ...func(log.CtxLogger, Node) log.CtxLogger) { ln, ok := n.(LoggingNode) if !ok { return } - nt := reflect.TypeOf(ln) - for nt.Kind() == reflect.Ptr { - nt = nt.Elem() + if len(options) == 0 { + // default options + logger = LoggerWithNodeID(logger, n) + logger = LoggerWithComponent(logger, n) + } else { + for _, c := range options { + logger = c(logger, n) + } } + ln.SetLogger(logger) +} - logger = logger.WithComponent(nt.Name()) +// LoggerWithNodeID creates a logger with the node ID field. +func LoggerWithNodeID(logger log.CtxLogger, n Node) log.CtxLogger { logger.Logger = logger.With().Str(log.NodeIDField, n.ID()).Logger() + return logger +} - ln.SetLogger(logger) +// LoggerWithComponent creates a logger with the component set to the node name. +func LoggerWithComponent(logger log.CtxLogger, v Node) log.CtxLogger { + t := reflect.TypeOf(v) + for t.Kind() == reflect.Ptr { + t = t.Elem() + } + + logger = logger.WithComponent(t.Name()) + return logger } diff --git a/pkg/pipeline/stream/message.go b/pkg/pipeline/stream/message.go index 0b055d4b6..d12717bed 100644 --- a/pkg/pipeline/stream/message.go +++ b/pkg/pipeline/stream/message.go @@ -44,7 +44,9 @@ var ( ErrUnexpectedMessageStatus = cerrors.New("unexpected message status") ) -// Message represents a single message flowing through a pipeline. +// Message represents a single message flowing through a pipeline. Only a single +// node is allowed to hold a message and access its fields at a specific point +// in time, otherwise we could introduce race conditions. type Message struct { // Ctx is the context in which the record was fetched. It should be used for // any function calls when processing the message. If the context is done @@ -272,6 +274,19 @@ func (m *Message) Status() MessageStatus { } } +// StatusError returns the error that was returned when the message was acked or +// nacked. If the message was successfully acked/nacked or it is still open the +// method returns nil. +func (m *Message) StatusError() error { + switch m.Status() { + case MessageStatusAcked: + return m.Ack() + case MessageStatusNacked: + return m.Nack(nil, "") + } + return nil +} + // OpenMessagesTracker allows you to track messages until they reach the end of // the pipeline. type OpenMessagesTracker sync.WaitGroup diff --git a/pkg/pipeline/stream/parallel.go b/pkg/pipeline/stream/parallel.go new file mode 100644 index 000000000..0eec006d8 --- /dev/null +++ b/pkg/pipeline/stream/parallel.go @@ -0,0 +1,367 @@ +// Copyright © 2023 Meroxa, Inc. +// +// Licensed 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 stream + +import ( + "context" + "sync" + + "github.com/conduitio/conduit/pkg/foundation/cchan" + "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/conduitio/conduit/pkg/foundation/log" +) + +// ParallelNode wraps a PubSubNode and parallelizes it by running multiple +// instances of the node in separate worker goroutines. It also spawns a +// coordinator goroutine that is responsible for collecting the results and +// forwarding them to the next node down the line while maintaining the order of +// messages. +type ParallelNode struct { + Name string + // NewNode is the constructor of the wrapped PubSubNode, it should create + // the i-th node (useful for distinguishing nodes in logs). + NewNode func(i int) PubSubNode + Workers int + + base pubSubNodeBase + logger log.CtxLogger +} + +func (n *ParallelNode) ID() string { + return n.Name +} + +// Run is continuously fetching messages from the incoming channel (i.e. from +// the previous node) and submitting jobs to the job channel shared by all +// worker goroutines. Once a worker picks up the job, the same job is also sent +// to the coordinator which starts waiting for the job to be done. The worker is +// responsible for forwarding the message to the wrapped PubSubNode, waiting for +// it to process the message and then mark the job as done. Once the jobs are +// done the results are picked up by the coordinator which decides if the +// message should be sent to the next node, if it was filtered out or if an +// error happened and the node needs to stop running. The coordinator collects +// job results in the same order as the order of the dispatched jobs, so the +// order of messages is maintained. +func (n *ParallelNode) Run(ctx context.Context) error { + // allow each worker to store an error in the channel + errs := make(chan error, n.Workers) + trigger, cleanup, err := n.base.Trigger(ctx, n.logger, errs) + if err != nil { + return err + } + defer cleanup() + + // donePool is a pool of reusable channels for parallelNodeJob.Done, this + // allows us to keep allocations low + donePool := &sync.Pool{ + New: func() any { return make(chan struct{}) }, + } + + // workerJobs is the channel where workers take jobs from, it is not + // buffered, so it blocks when all workers are busy + workerJobs := make(chan parallelNodeJob) + var workerWg sync.WaitGroup + for i := 0; i < n.Workers; i++ { + node := n.NewNode(i) + worker := newParallelNodeWorker(node, workerJobs, n.logger) + workerWg.Add(1) + go func() { + defer workerWg.Done() + worker.Run(ctx) + }() + } + + // coordinatorJobs is the channel where coordinator takes the jobs from, + // it has a buffer, so it can store one job for each worker + coordinatorJobs := make(chan parallelNodeJob, n.Workers) + var coordinatorWg sync.WaitGroup + coordinatorWg.Add(1) + coordinator := newParallelNodeCoordinator(n.ID(), coordinatorJobs, errs, n.logger, n.base.Send, donePool) + go func() { + defer coordinatorWg.Done() + coordinator.Run(ctx) + }() + + // workersDone is closed once all workers stop running (for whatever reason) + workersDone := make(chan struct{}) + go func() { + workerWg.Wait() + close(workersDone) + }() + + defer func() { + close(workerJobs) + close(coordinatorJobs) + workerWg.Wait() + coordinatorWg.Wait() + for { + select { + case workerErr := <-errs: + err = cerrors.LogOrReplace(err, workerErr, func() { + n.logger.Warn(ctx).Err(workerErr).Msg("parallel worker node failed") + }) + default: + return + } + } + }() + + for { + msg, err := trigger() + if err != nil || msg == nil { + return err + } + + job := parallelNodeJob{ + Message: msg, + done: donePool.Get().(chan struct{}), + } + + // try sending the job to a worker + select { + case workerJobs <- job: + // we submitted the job to a worker, give it to the coordinator as well + coordinatorJobs <- job + case <-workersDone: + // no worker is running anymore, they must have all failed, nack the + // message and stop running + noWorkerRunningErr := cerrors.New("no worker is running") + err = msg.Nack(noWorkerRunningErr, n.ID()) + if err != nil { + return err + } + return noWorkerRunningErr + } + } +} + +func (n *ParallelNode) Sub(in <-chan *Message) { + n.base.Sub(in) +} + +func (n *ParallelNode) Pub() <-chan *Message { + return n.base.Pub() +} + +func (n *ParallelNode) SetLogger(logger log.CtxLogger) { + n.logger = logger +} + +// parallelNodeJob is a single job processed by a worker. Once the job is +// processed, the worker needs to call Done to signal that it has finished. The +// coordinator calls Wait to wait for the job to be processed. +type parallelNodeJob struct { + Message *Message + done chan struct{} +} + +// Wait blocks until Done is called. +func (j parallelNodeJob) Wait() { + <-j.done +} + +// Done signals that the job is done. It blocks until another goroutine calls +// Wait (the coordinator goroutine). +func (j parallelNodeJob) Done() { + j.done <- struct{}{} +} + +// parallelNodeCoordinator coordinates the messages that are processed by +// workers and ensures their order. +type parallelNodeCoordinator struct { + name string + jobs <-chan parallelNodeJob + errs chan<- error + logger log.CtxLogger + send func(ctx context.Context, logger log.CtxLogger, msg *Message) error + donePool *sync.Pool +} + +func newParallelNodeCoordinator( + name string, + jobs <-chan parallelNodeJob, + errs chan<- error, + logger log.CtxLogger, + send func(ctx context.Context, logger log.CtxLogger, msg *Message) error, + donePool *sync.Pool, +) *parallelNodeCoordinator { + logger.Logger = logger.With().Str(log.ParallelWorkerIDField, name+"-coordinator").Logger() + return ¶llelNodeCoordinator{ + name: name, + jobs: jobs, + errs: errs, + logger: logger, + send: send, + donePool: donePool, + } +} + +func (c *parallelNodeCoordinator) Run(ctx context.Context) { + // fail toggles a short circuit that causes all future messages to be nacked + fail := false + for job := range c.jobs { + // wait for job to be done, this ensures that the order is maintained + job.Wait() + // put the channel back into the pool + c.donePool.Put(job.done) + + // Check if the message was successfully processed or not. There are two + // cases when the status error wouldn't be nil: + // - If the message is acknowledged by a processor (i.e. filtered out) + // and we fail to deliver the acknowledgment to the connector. + // - If the processor failed to process the message and nacked it but + // the nack failed (i.e. it wasn't stored in the DLQ). + // In both cases the processor node failed and returned an error, so we + // need to propagate the error and nack all following messages. + err := job.Message.StatusError() + if err != nil { + // propagate error to main node and trigger short circuit, all + // messages need to fail from here on out + c.errs <- err + fail = true + continue + } + if job.Message.Status() != MessageStatusOpen { + // message already acked or nacked + continue + } + if fail { + err = job.Message.Nack(cerrors.Errorf("another message failed to be processed successfully"), c.name) + if err != nil { + c.errs <- err + } + continue + } + err = c.send(ctx, c.logger, job.Message) + if err != nil { + // could not send message to next node, this means the context is + // cancelled, nack the message and drain jobs channel + err = job.Message.Nack(err, c.name) + if err != nil { + c.errs <- err + fail = true + } + } + } +} + +// parallelNodeWorker runs the two goroutines, one for the worker itself and one +// for the forwarder. The worker is in charge of processing the node, the +// forwarder accepts jobs, forwards them to the node and then waits for it to +// either ack/nack the message or send it to its output channel. Once the +// message has been processed it forwards it to the coordinator. +type parallelNodeWorker struct { + node PubSubNode + jobs cchan.ChanOut[parallelNodeJob] + logger log.CtxLogger +} + +func newParallelNodeWorker( + node PubSubNode, + jobs <-chan parallelNodeJob, + logger log.CtxLogger, +) *parallelNodeWorker { + nodeLogger := logger + nodeLogger.Logger = logger.With().Str(log.ParallelWorkerIDField, node.ID()).Logger() + SetLogger(node, nodeLogger, LoggerWithComponent) + + logger.Logger = logger.With().Str(log.ParallelWorkerIDField, node.ID()+"-worker").Logger() + return ¶llelNodeWorker{ + node: node, + jobs: jobs, + logger: logger, + } +} + +func (w *parallelNodeWorker) Run(ctx context.Context) { + in := make(chan *Message) + w.node.Sub(in) + out := w.node.Pub() + + workerDone := make(chan struct{}) + go func() { + defer close(workerDone) + w.runWorker(ctx) + }() + + forwarderDone := make(chan struct{}) + go func() { + defer func() { + <-workerDone + close(forwarderDone) + }() + w.runForwarder(in, out) + }() + + <-forwarderDone +} + +func (w *parallelNodeWorker) runWorker(ctx context.Context) { + // we can ignore errors, if an error happens they are propagated through + // a message nack/ack to the forwarder node and further to the coordinator + _ = w.node.Run(ctx) +} + +func (w *parallelNodeWorker) runForwarder(in chan<- *Message, out <-chan *Message) { + defer close(in) + for job := range w.jobs { + ctx := job.Message.Ctx + select { + case _, ok := <-out: + if ok { + panic("worker node produced a message without receiving a message") + } + // out is closed, worker node stopped running (closed context?), + // nack in-flight message and ignore error, it will be picked up + // by the coordinator + _ = job.Message.Nack(cerrors.New("worker not running"), w.node.ID()) + job.Done() + return + case in <- job.Message: + // message submitted to worker node + w.logger.Trace(ctx).Msg("message sent to worker") + } + + select { + case <-job.Message.Acked(): + // message was acked, i.e. filtered out + w.logger.Trace(ctx).Msg("worker acked the message") + case <-job.Message.Nacked(): + // message was nacked, i.e. sent to DLQ or dropped + w.logger.Trace(ctx).Msg("worker nacked the message") + case _, ok := <-out: + // message was processed + if ok { + w.logger.Trace(ctx).Msg("worker successfully processed the message") + } else { + w.logger.Trace(ctx).Msg("worker stopped running") + } + } + + // get error before we give the message to the coordinator + // if the message status returns an error it means the worker stopped + // running and the error will be propagated by the coordinator + err := job.Message.StatusError() + + // give message to coordinator + job.Done() + + if err != nil { + // message processing failed, worker stopped running, stop accepting jobs + w.logger.Warn(ctx).Err(err).Msg("worker stopped running, stopping forwarder") + return + } + } +} diff --git a/pkg/pipeline/stream/parallel_test.go b/pkg/pipeline/stream/parallel_test.go new file mode 100644 index 000000000..5c4e700f8 --- /dev/null +++ b/pkg/pipeline/stream/parallel_test.go @@ -0,0 +1,617 @@ +// Copyright © 2023 Meroxa, Inc. +// +// Licensed 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 stream + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/conduitio/conduit/pkg/foundation/cchan" + "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/conduitio/conduit/pkg/foundation/csync" + "github.com/conduitio/conduit/pkg/foundation/log" + "github.com/conduitio/conduit/pkg/foundation/metrics/noop" + pmock "github.com/conduitio/conduit/pkg/processor/mock" + "github.com/conduitio/conduit/pkg/record" + "github.com/golang/mock/gomock" + "github.com/matryer/is" +) + +type parallelTestNode struct { + Name string + F func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error + + pub chan *Message + sub <-chan *Message +} + +func (n *parallelTestNode) ID() string { return n.Name } +func (n *parallelTestNode) Pub() <-chan *Message { + n.pub = make(chan *Message) + return n.pub +} +func (n *parallelTestNode) Sub(in <-chan *Message) { + n.sub = in +} +func (n *parallelTestNode) Run(ctx context.Context) error { + return n.F(ctx, n.sub, n.pub) +} + +func TestParallelNode_Worker(t *testing.T) { + logger := log.Nop() + ctx := context.Background() + + testError := cerrors.New("test error") + + testCases := []struct { + name string + nodeFunc func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error + wantStatus MessageStatus + wantStatusError error + }{{ + name: "open", + nodeFunc: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + pub <- msg + } + }, + wantStatus: MessageStatusOpen, + }, { + name: "ack", + nodeFunc: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + err = msg.Ack() + if err != nil { + return err + } + } + }, + wantStatus: MessageStatusAcked, + }, { + name: "nack", + nodeFunc: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + msg.RegisterNackHandler(func(*Message, NackMetadata) error { + // register nack handler so nack can succeed + return nil + }) + err = msg.Nack(testError, "test-node") + if err != nil { + return err + } + } + }, + wantStatus: MessageStatusNacked, + wantStatusError: nil, + }, { + name: "failed ack", + nodeFunc: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + msg.RegisterAckHandler(func(*Message) error { + // register failing ack handler so ack fails + return testError + }) + err = msg.Ack() + if err != nil { + return err + } + } + }, + wantStatus: MessageStatusAcked, + wantStatusError: testError, + }, { + name: "failed nack", + nodeFunc: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + err = msg.Nack(testError, "test-node") + if err != nil { + return err + } + } + }, + wantStatus: MessageStatusNacked, + wantStatusError: testError, + }} + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + is := is.New(t) + n := ¶llelTestNode{ + Name: "test-node", + F: tc.nodeFunc, + } + + jobs := make(chan parallelNodeJob) + worker := newParallelNodeWorker(n, jobs, logger) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + worker.Run(ctx) + }() + + job := parallelNodeJob{ + Message: &Message{}, + done: make(chan struct{}), + } + jobs <- job + + _, ok, err := cchan.ChanOut[struct{}](job.done).RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(ok) + + is.Equal(job.Message.Status(), tc.wantStatus) + is.True(cerrors.Is(job.Message.StatusError(), tc.wantStatusError)) + + close(jobs) + + err = (*csync.WaitGroup)(&wg).WaitTimeout(ctx, time.Second) + is.NoErr(err) + }) + } +} + +func TestParallelNode_Coordinator(t *testing.T) { + logger := log.Nop() + ctx := context.Background() + + testError := cerrors.New("test error") + + testCases := []struct { + name string + processMessage func(*Message) + wantMessage bool + wantError error + }{{ + name: "open", + processMessage: func(*Message) { /* do nothing */ }, + wantMessage: true, + wantError: nil, + }, { + name: "ack", + processMessage: func(msg *Message) { + _ = msg.Ack() + }, + wantMessage: false, + wantError: nil, + }, { + name: "nack", + processMessage: func(msg *Message) { + msg.RegisterNackHandler(func(*Message, NackMetadata) error { + // register nack handler so nack can succeed + return nil + }) + _ = msg.Nack(testError, "test-node") + }, + wantMessage: false, + wantError: nil, + }, { + name: "failed ack", + processMessage: func(msg *Message) { + msg.RegisterAckHandler(func(*Message) error { + // register failing ack handler so ack fails + return testError + }) + _ = msg.Ack() + }, + wantMessage: false, + wantError: testError, + }, { + name: "failed nack", + processMessage: func(msg *Message) { + _ = msg.Nack(testError, "test-node") + }, + wantMessage: false, + wantError: testError, + }} + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + is := is.New(t) + + jobs := make(chan parallelNodeJob) + errs := make(chan error) + out := make(chan *Message) + send := func(ctx context.Context, _ log.CtxLogger, msg *Message) error { + return cchan.ChanIn[*Message](out).Send(ctx, msg) + } + donePool := &sync.Pool{ + New: func() any { return make(chan struct{}) }, + } + coordinator := newParallelNodeCoordinator("test-node", jobs, errs, logger, send, donePool) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + coordinator.Run(ctx) + }() + + job := parallelNodeJob{ + Message: &Message{}, + done: donePool.Get().(chan struct{}), + } + jobs <- job + + tc.processMessage(job.Message) + + job.Done() + + _, ok, err := cchan.ChanOut[*Message](out).RecvTimeout(ctx, time.Millisecond*10) + if tc.wantMessage { + is.NoErr(err) + is.True(ok) + } else { + is.True(cerrors.Is(err, context.DeadlineExceeded)) + } + + gotErr, ok, err := cchan.ChanOut[error](errs).RecvTimeout(ctx, time.Millisecond*10) + if tc.wantError != nil { + is.NoErr(err) + is.True(ok) + is.True(cerrors.Is(gotErr, tc.wantError)) + } else { + is.True(cerrors.Is(err, context.DeadlineExceeded)) + } + + // regardless of what happened above we should be able to send another job in + job = parallelNodeJob{ + Message: &Message{}, + done: donePool.Get().(chan struct{}), + } + jobs <- job + job.Done() + + _, ok, err = cchan.ChanOut[*Message](out).RecvTimeout(ctx, time.Millisecond*10) + // if we did not expect an error in the previous message then this one should go through + if tc.wantError == nil { + is.NoErr(err) + is.True(ok) + } else { + is.True(cerrors.Is(err, context.DeadlineExceeded)) + } + + gotErr, ok, err = cchan.ChanOut[error](errs).RecvTimeout(ctx, time.Millisecond*10) + // if we expected an error above we expect one for all following messages + if tc.wantError != nil { + is.NoErr(err) + is.True(ok) + is.True(gotErr != nil) + } else { + is.True(cerrors.Is(err, context.DeadlineExceeded)) + } + + // closing jobs should stop the coordinator + close(jobs) + + err = (*csync.WaitGroup)(&wg).WaitTimeout(ctx, time.Second) + is.NoErr(err) + }) + } +} + +func TestParallelNode_Success(t *testing.T) { + is := is.New(t) + logger := log.Nop() + ctx := context.Background() + controlChan := cchan.Chan[struct{}](make(chan struct{})) + + const workerCount = 10 + + newPubSubNode := func(i int) PubSubNode { + return ¶llelTestNode{ + Name: fmt.Sprintf("test-node-%d", i), + F: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + controlChan <- struct{}{} + pub <- msg + } + }, + } + } + + pn := &ParallelNode{ + NewNode: newPubSubNode, + Workers: workerCount, + } + pn.SetLogger(logger) + + out := pn.Pub() + in := make(chan *Message) + pn.Sub(in) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := pn.Run(ctx) + is.NoErr(err) + }() + + // send messages to workers until each has one message + 1 for the parallel node itself + for i := 0; i < workerCount+1; i++ { + in <- &Message{Ctx: ctx, Record: record.Record{Key: record.StructuredData{"id": i}}} + } + + // sending another message to the node results in a timeout, because it is blocking + err := cchan.ChanIn[*Message](in).SendTimeout(ctx, &Message{}, time.Millisecond*100) + is.True(cerrors.Is(err, context.DeadlineExceeded)) + + // unblock all workers and allow them to send the messages forward + for i := 0; i < workerCount+1; i++ { + _, ok, err := controlChan.RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(ok) + } + + // stop node + close(in) + + // ensure messages are received in order and out is closed + i := 0 + for { + msg, ok, err := cchan.ChanOut[*Message](out).RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + if !ok { + break + } + is.Equal(msg.Record.Key, record.StructuredData{"id": i}) + i++ + } + is.Equal(i, workerCount+1) + + err = (*csync.WaitGroup)(&wg).WaitTimeout(ctx, time.Second) + is.NoErr(err) +} + +func TestParallelNode_ErrorAll(t *testing.T) { + is := is.New(t) + logger := log.Nop() + ctx := context.Background() + controlChan := cchan.Chan[struct{}](make(chan struct{})) + + const workerCount = 10 + + newPubSubNode := func(i int) PubSubNode { + name := fmt.Sprintf("test-node-%d", i) + return ¶llelTestNode{ + Name: name, + F: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + is.NoErr(err) + is.True(ok) + controlChan <- struct{}{} + return msg.Nack(cerrors.New("test error"), name) + }, + } + } + + pn := &ParallelNode{ + NewNode: newPubSubNode, + Workers: workerCount, + } + pn.SetLogger(logger) + + out := pn.Pub() + in := make(chan *Message) + pn.Sub(in) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := pn.Run(ctx) + is.True(err != nil) + }() + + // send messages to workers until each has one message + for i := 0; i < workerCount; i++ { + in <- &Message{Ctx: ctx, Record: record.Record{Key: record.StructuredData{"id": i}}} + } + // unblock all workers and allow them to send the messages forward + for i := 0; i < workerCount; i++ { + _, ok, err := controlChan.RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(ok) + } + + // ensure out gets closed + _, ok, err := cchan.ChanOut[*Message](out).RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(!ok) + + err = (*csync.WaitGroup)(&wg).WaitTimeout(ctx, time.Second) + is.NoErr(err) +} + +func TestParallelNode_ErrorSingle(t *testing.T) { + is := is.New(t) + logger := log.Nop() + ctx := context.Background() + controlChan := cchan.Chan[struct{}](make(chan struct{})) + + const workerCount = 10 + + newPubSubNode := func(i int) PubSubNode { + name := fmt.Sprintf("test-node-%d", i) + return ¶llelTestNode{ + Name: name, + F: func(ctx context.Context, sub <-chan *Message, pub chan<- *Message) error { + defer close(pub) + for { + msg, ok, err := cchan.ChanOut[*Message](sub).Recv(ctx) + if !ok { + return err + } + controlChan <- struct{}{} + if msg.Record.Key.(record.StructuredData)["id"].(int) == 1 { + // only message with id 1 fails + return msg.Nack(cerrors.New("test error"), name) + } + pub <- msg + } + }, + } + } + + pn := &ParallelNode{ + NewNode: newPubSubNode, + Workers: workerCount, + } + pn.SetLogger(logger) + + out := pn.Pub() + in := make(chan *Message) + pn.Sub(in) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := pn.Run(ctx) + is.True(err != nil) + }() + + // send messages to workers until each has one message + for i := 0; i < workerCount; i++ { + in <- &Message{Ctx: ctx, Record: record.Record{Key: record.StructuredData{"id": i}}} + } + // unblock all workers and allow them to send the messages forward + for i := 0; i < workerCount; i++ { + _, ok, err := controlChan.RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(ok) + } + + // one message should get through + msg, ok, err := cchan.ChanOut[*Message](out).RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(ok) + is.Equal(msg.Record.Key, record.StructuredData{"id": 0}) + + // out should get closed after that, as all other messages are nacked + _, ok, err = cchan.ChanOut[*Message](out).RecvTimeout(ctx, time.Millisecond*100) + is.NoErr(err) + is.True(!ok) + + err = (*csync.WaitGroup)(&wg).WaitTimeout(ctx, time.Second) + is.NoErr(err) +} + +func TestParallelNode_Processor(t *testing.T) { + is := is.New(t) + logger := log.Nop() + ctx := context.Background() + ctrl := gomock.NewController(t) + + var workersCurrent int32 // current number of parallel workers + var workersHighWatermark int32 // highest number of parallel workers + + const workerCount = 100 + const msgCount = 1000 + + // create a dummy processor + proc := pmock.NewProcessor(ctrl) + proc.EXPECT().Process(gomock.Any(), gomock.Any()).DoAndReturn(func(ctx context.Context, r record.Record) (record.Record, error) { + current := atomic.AddInt32(&workersCurrent, 1) + atomic.CompareAndSwapInt32(&workersHighWatermark, current-1, current) + time.Sleep(time.Millisecond) // sleep to let other workers catch up + atomic.AddInt32(&workersCurrent, -1) + return r, nil + }).Times(msgCount) + + newProcNode := func(i int) PubSubNode { + return &ProcessorNode{ + Name: fmt.Sprintf("test-%d", i), + Processor: proc, + ProcessorTimer: noop.Timer{}, + } + } + + pn := &ParallelNode{ + NewNode: newProcNode, + Workers: workerCount, + } + pn.SetLogger(logger) + + out := pn.Pub() + in := make(chan *Message) + pn.Sub(in) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := pn.Run(ctx) + is.NoErr(err) + }() + + wg.Add(1) + go func() { + defer wg.Done() + for i := 0; i < msgCount; i++ { + in <- &Message{Ctx: ctx, Record: record.Record{Key: record.StructuredData{"id": i}}} + } + close(in) + }() + + i := 0 + for msg := range out { + is.Equal(msg.Record.Key, record.StructuredData{"id": i}) + i++ + } + is.Equal(i, msgCount) + + err := (*csync.WaitGroup)(&wg).WaitTimeout(ctx, time.Second) + is.NoErr(err) + is.True(workersHighWatermark <= workerCount) // more workers spawned than expected + is.True(workersHighWatermark >= 2) // no parallel workers spawned +} diff --git a/pkg/processor/instance.go b/pkg/processor/instance.go index 53753f7b7..46b65e810 100644 --- a/pkg/processor/instance.go +++ b/pkg/processor/instance.go @@ -83,4 +83,5 @@ type Parent struct { // Config holds configuration data for building a processor. type Config struct { Settings map[string]string + Workers int } diff --git a/pkg/processor/service.go b/pkg/processor/service.go index 8579fbe22..b6da79feb 100644 --- a/pkg/processor/service.go +++ b/pkg/processor/service.go @@ -92,6 +92,13 @@ func (s *Service) Create( cfg Config, pt ProvisionType, ) (*Instance, error) { + if cfg.Workers < 0 { + return nil, cerrors.New("processor workers can't be negative") + } + if cfg.Workers == 0 { + cfg.Workers = 1 + } + builder, err := s.registry.Get(procType) if err != nil { return nil, err diff --git a/pkg/processor/service_test.go b/pkg/processor/service_test.go index 166db0f8f..3310cbee1 100644 --- a/pkg/processor/service_test.go +++ b/pkg/processor/service_test.go @@ -18,7 +18,6 @@ import ( "context" "testing" - "github.com/conduitio/conduit/pkg/foundation/assert" "github.com/conduitio/conduit/pkg/foundation/cerrors" "github.com/conduitio/conduit/pkg/foundation/database/inmemory" dbmock "github.com/conduitio/conduit/pkg/foundation/database/mock" @@ -31,6 +30,7 @@ import ( ) func TestService_Init_Success(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} ctrl := gomock.NewController(t) @@ -38,19 +38,19 @@ func TestService_Init_Success(t *testing.T) { procType := "processor-type" p := mock.NewProcessor(ctrl) - registry := newTestBuilderRegistry(t, map[string]processor.Interface{procType: p}) + registry := newTestBuilderRegistry(is, map[string]processor.Interface{procType: p}) service := processor.NewService(log.Nop(), db, registry) // create a processor instance _, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) want := service.List(ctx) // create a new processor service and initialize it service = processor.NewService(log.Nop(), db, registry) err = service.Init(ctx) - assert.Ok(t, err) + is.NoErr(err) got := service.List(ctx) @@ -58,8 +58,8 @@ func TestService_Init_Success(t *testing.T) { got[k].UpdatedAt = want[k].UpdatedAt got[k].CreatedAt = want[k].CreatedAt } - assert.Equal(t, want, got) - assert.Equal(t, len(got), 1) + is.Equal(want, got) + is.Equal(len(got), 1) } func TestService_Check(t *testing.T) { @@ -71,14 +71,8 @@ func TestService_Check(t *testing.T) { name string wantErr error }{ - { - name: "db ok", - wantErr: nil, - }, - { - name: "db not ok", - wantErr: cerrors.New("db is under the weather today"), - }, + {name: "db ok", wantErr: nil}, + {name: "db not ok", wantErr: cerrors.New("db error")}, } for _, tc := range testCases { @@ -95,6 +89,7 @@ func TestService_Check(t *testing.T) { } func TestService_Create_Success(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} ctrl := gomock.NewController(t) @@ -113,22 +108,24 @@ func TestService_Create_Success(t *testing.T) { "processor-config-field-1": "foo", "processor-config-field-2": "bar", }, + Workers: 1, }, Processor: p, } - registry := newTestBuilderRegistry(t, map[string]processor.Interface{want.Type: p}) + registry := newTestBuilderRegistry(is, map[string]processor.Interface{want.Type: p}) service := processor.NewService(log.Nop(), db, registry) got, err := service.Create(ctx, want.ID, want.Type, want.Parent, want.Config, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) want.ID = got.ID // uuid is random want.CreatedAt = got.CreatedAt want.UpdatedAt = got.UpdatedAt - assert.Equal(t, want, got) + is.Equal(want, got) } func TestService_Create_BuilderNotFound(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} @@ -144,11 +141,12 @@ func TestService_Create_BuilderNotFound(t *testing.T) { processor.ProvisionTypeAPI, ) - assert.Error(t, err) - assert.Nil(t, got) + is.True(err != nil) + is.Equal(got, nil) } func TestService_Create_BuilderFail(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} @@ -162,7 +160,7 @@ func TestService_Create_BuilderFail(t *testing.T) { return nil, wantErr }, ) - assert.Ok(t, err) + is.NoErr(err) service := processor.NewService(log.Nop(), db, registry) @@ -174,11 +172,32 @@ func TestService_Create_BuilderFail(t *testing.T) { processor.Config{}, processor.ProvisionTypeAPI, ) - assert.True(t, cerrors.Is(err, wantErr), "expected builder error") - assert.Nil(t, got) + is.True(cerrors.Is(err, wantErr)) // expected builder error + is.Equal(got, nil) +} + +func TestService_Create_WorkersNegative(t *testing.T) { + is := is.New(t) + ctx := context.Background() + db := &inmemory.DB{} + registry := processor.NewBuilderRegistry() + + service := processor.NewService(log.Nop(), db, registry) + + got, err := service.Create( + ctx, + uuid.NewString(), + "processor-type", + processor.Parent{}, + processor.Config{}, + processor.ProvisionTypeAPI, + ) + is.True(err != nil) // expected workers error + is.Equal(got, nil) } func TestService_Delete_Success(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} ctrl := gomock.NewController(t) @@ -187,31 +206,33 @@ func TestService_Delete_Success(t *testing.T) { p := mock.NewProcessor(ctrl) p.EXPECT().Close() - registry := newTestBuilderRegistry(t, map[string]processor.Interface{procType: p}) + registry := newTestBuilderRegistry(is, map[string]processor.Interface{procType: p}) service := processor.NewService(log.Nop(), db, registry) // create a processor instance i, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) err = service.Delete(ctx, i.ID) - assert.Ok(t, err) + is.NoErr(err) got, err := service.Get(ctx, i.ID) - assert.True(t, cerrors.Is(err, processor.ErrInstanceNotFound), "expected instance not found error") - assert.Nil(t, got) + is.True(cerrors.Is(err, processor.ErrInstanceNotFound)) // expected instance not found error + is.Equal(got, nil) } func TestService_Delete_Fail(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} service := processor.NewService(log.Nop(), db, processor.NewBuilderRegistry()) err := service.Delete(ctx, "non-existent processor") - assert.True(t, cerrors.Is(err, processor.ErrInstanceNotFound), "expected instance not found error") + is.True(cerrors.Is(err, processor.ErrInstanceNotFound)) // expected instance not found error } func TestService_Get_Success(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} ctrl := gomock.NewController(t) @@ -219,39 +240,42 @@ func TestService_Get_Success(t *testing.T) { procType := "processor-type" p := mock.NewProcessor(ctrl) - registry := newTestBuilderRegistry(t, map[string]processor.Interface{procType: p}) + registry := newTestBuilderRegistry(is, map[string]processor.Interface{procType: p}) service := processor.NewService(log.Nop(), db, registry) // create a processor instance want, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) got, err := service.Get(ctx, want.ID) - assert.Ok(t, err) - assert.Equal(t, want, got) + is.NoErr(err) + is.Equal(want, got) } func TestService_Get_Fail(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} service := processor.NewService(log.Nop(), db, processor.NewBuilderRegistry()) got, err := service.Get(ctx, "non-existent processor") - assert.True(t, cerrors.Is(err, processor.ErrInstanceNotFound), "expected instance not found error") - assert.Nil(t, got) + is.True(cerrors.Is(err, processor.ErrInstanceNotFound)) // expected instance not found error + is.Equal(got, nil) } func TestService_List_Empty(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} service := processor.NewService(log.Nop(), db, processor.NewBuilderRegistry()) instances := service.List(ctx) - assert.NotNil(t, instances) - assert.True(t, len(instances) == 0, "expected an empty map") + is.True(instances != nil) + is.True(len(instances) == 0) // expected an empty map } func TestService_List_Some(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} ctrl := gomock.NewController(t) @@ -259,22 +283,23 @@ func TestService_List_Some(t *testing.T) { procType := "processor-type" p := mock.NewProcessor(ctrl) - registry := newTestBuilderRegistry(t, map[string]processor.Interface{procType: p}) + registry := newTestBuilderRegistry(is, map[string]processor.Interface{procType: p}) service := processor.NewService(log.Nop(), db, registry) // create a couple of processor instances i1, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) i2, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) i3, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) instances := service.List(ctx) - assert.Equal(t, map[string]*processor.Instance{i1.ID: i1, i2.ID: i2, i3.ID: i3}, instances) + is.Equal(map[string]*processor.Instance{i1.ID: i1, i2.ID: i2, i3.ID: i3}, instances) } func TestService_Update_Success(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} ctrl := gomock.NewController(t) @@ -282,12 +307,12 @@ func TestService_Update_Success(t *testing.T) { procType := "processor-type" p := mock.NewProcessor(ctrl) - registry := newTestBuilderRegistry(t, map[string]processor.Interface{procType: p}) + registry := newTestBuilderRegistry(is, map[string]processor.Interface{procType: p}) service := processor.NewService(log.Nop(), db, registry) // create a processor instance want, err := service.Create(ctx, uuid.NewString(), procType, processor.Parent{}, processor.Config{}, processor.ProvisionTypeAPI) - assert.Ok(t, err) + is.NoErr(err) newConfig := processor.Config{ Settings: map[string]string{ @@ -297,29 +322,30 @@ func TestService_Update_Success(t *testing.T) { } got, err := service.Update(ctx, want.ID, newConfig) - assert.Ok(t, err) - assert.Equal(t, want, got) // same instance is returned - assert.Equal(t, newConfig, got.Config) // config was updated + is.NoErr(err) + is.Equal(want, got) // same instance is returned + is.Equal(newConfig, got.Config) // config was updated got, err = service.Get(ctx, want.ID) - assert.Ok(t, err) - assert.Equal(t, newConfig, got.Config) + is.NoErr(err) + is.Equal(newConfig, got.Config) } func TestService_Update_Fail(t *testing.T) { + is := is.New(t) ctx := context.Background() db := &inmemory.DB{} service := processor.NewService(log.Nop(), db, processor.NewBuilderRegistry()) got, err := service.Update(ctx, "non-existent processor", processor.Config{}) - assert.True(t, cerrors.Is(err, processor.ErrInstanceNotFound), "expected instance not found error") - assert.Nil(t, got) + is.True(cerrors.Is(err, processor.ErrInstanceNotFound)) // expected instance not found error + is.Equal(got, nil) } // newTestBuilderRegistry creates a registry with builders for the supplied // processors map keyed by processor type. If a value in the map is nil then a // builder will be registered that returns an error. -func newTestBuilderRegistry(t *testing.T, processors map[string]processor.Interface) *processor.BuilderRegistry { +func newTestBuilderRegistry(is *is.I, processors map[string]processor.Interface) *processor.BuilderRegistry { registry := processor.NewBuilderRegistry() for procType, p := range processors { err := registry.Register( @@ -331,7 +357,7 @@ func newTestBuilderRegistry(t *testing.T, processors map[string]processor.Interf return nil, cerrors.New("builder error") }, ) - assert.Ok(t, err) + is.NoErr(err) } return registry } diff --git a/pkg/provisioning/enrich.go b/pkg/provisioning/enrich.go index 552645206..cd2c11db2 100644 --- a/pkg/provisioning/enrich.go +++ b/pkg/provisioning/enrich.go @@ -50,6 +50,9 @@ func enrichConnectorsConfig(mp map[string]ConnectorConfig, pipelineID string) ma func enrichProcessorsConfig(mp map[string]ProcessorConfig, parentID string) map[string]ProcessorConfig { newMap := make(map[string]ProcessorConfig, len(mp)) for k, cfg := range mp { + if cfg.Workers == 0 { + cfg.Workers = 1 + } newID := parentID + ":" + k newMap[newID] = cfg } diff --git a/pkg/provisioning/enrich_test.go b/pkg/provisioning/enrich_test.go index 1ee3d58dc..3c1828ccb 100644 --- a/pkg/provisioning/enrich_test.go +++ b/pkg/provisioning/enrich_test.go @@ -35,7 +35,8 @@ func TestEnrich_DefaultValues(t *testing.T) { }, Processors: map[string]ProcessorConfig{ "proc2": { - Type: "js", + Type: "js", + Workers: 2, Settings: map[string]string{ "additionalProp1": "string", }, @@ -72,7 +73,8 @@ func TestEnrich_DefaultValues(t *testing.T) { }, Processors: map[string]ProcessorConfig{ "pipeline1:con1:proc2": { - Type: "js", + Type: "js", + Workers: 2, Settings: map[string]string{ "additionalProp1": "string", }, @@ -82,7 +84,8 @@ func TestEnrich_DefaultValues(t *testing.T) { }, Processors: map[string]ProcessorConfig{ "pipeline1:proc1": { - Type: "js", + Type: "js", + Workers: 1, Settings: map[string]string{ "additionalProp1": "string", }, diff --git a/pkg/provisioning/parse.go b/pkg/provisioning/parse.go index 8710ab7f9..80e3cd46a 100644 --- a/pkg/provisioning/parse.go +++ b/pkg/provisioning/parse.go @@ -28,6 +28,7 @@ import ( type ProcessorConfig struct { Type string `yaml:"type"` Settings map[string]string `yaml:"settings"` + Workers int `yaml:"workers"` } type ConnectorConfig struct { diff --git a/pkg/provisioning/service.go b/pkg/provisioning/service.go index 4b688a0d0..0c8b048f0 100644 --- a/pkg/provisioning/service.go +++ b/pkg/provisioning/service.go @@ -423,6 +423,7 @@ func (s *Service) createConnector(ctx context.Context, pipelineID string, id str func (s *Service) createProcessor(ctx context.Context, parentID string, parentType processor.ParentType, id string, config ProcessorConfig) error { cfg := processor.Config{ Settings: config.Settings, + Workers: config.Workers, } parent := processor.Parent{ ID: parentID, diff --git a/pkg/provisioning/service_test.go b/pkg/provisioning/service_test.go index 43bad994b..5078ecbcc 100644 --- a/pkg/provisioning/service_test.go +++ b/pkg/provisioning/service_test.go @@ -63,7 +63,8 @@ var pipeline1 = PipelineConfig{ }, Processors: map[string]ProcessorConfig{ "pipeline1:con2:proc1con": { - Type: "js", + Type: "js", + Workers: 10, Settings: map[string]string{ "additionalProp1": "string", }, @@ -73,7 +74,8 @@ var pipeline1 = PipelineConfig{ }, Processors: map[string]ProcessorConfig{ "pipeline1:proc1": { - Type: "js", + Type: "js", + Workers: 1, Settings: map[string]string{ "additionalProp1": "string", }, @@ -156,8 +158,13 @@ func TestProvision_PipelineWithConnectorsAndProcessors(t *testing.T) { Name: pipeline1.Connectors["pipeline1:con2"].Name, Settings: pipeline1.Connectors["pipeline1:con2"].Settings, } - procCfg := processor.Config{ + procCfg1 := processor.Config{ Settings: pipeline1.Processors["pipeline1:proc1"].Settings, + Workers: pipeline1.Processors["pipeline1:proc1"].Workers, + } + procCfg2 := processor.Config{ + Settings: pipeline1.Connectors["pipeline1:con2"].Processors["pipeline1:con2:proc1con"].Settings, + Workers: pipeline1.Connectors["pipeline1:con2"].Processors["pipeline1:con2:proc1con"].Workers, } procParentPipeline := processor.Parent{ ID: pipeline1.Name, @@ -179,9 +186,9 @@ func TestProvision_PipelineWithConnectorsAndProcessors(t *testing.T) { connService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2", connector.TypeDestination, pipeline1.Connectors["pipeline1:con2"].Plugin, pipeline1.Name, cfg2, connector.ProvisionTypeConfig) pipelineService.EXPECT().AddConnector(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:con2") - procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con", "js", procParentConn, procCfg, processor.ProvisionTypeConfig) + procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con", "js", procParentConn, procCfg2, processor.ProvisionTypeConfig) connService.EXPECT().AddProcessor(gomock.Not(gomock.Nil()), "pipeline1:con2", "pipeline1:con2:proc1con") - procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:proc1", "js", procParentPipeline, procCfg, processor.ProvisionTypeConfig) + procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:proc1", "js", procParentPipeline, procCfg1, processor.ProvisionTypeConfig) pipelineService.EXPECT().AddProcessor(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:proc1") pipelineService.EXPECT().Start(gomock.Not(gomock.Nil()), connService, procService, plugService, pipeline1.Name) @@ -216,8 +223,13 @@ func TestProvision_Rollback(t *testing.T) { Name: pipeline1.Connectors["pipeline1:con2"].Name, Settings: pipeline1.Connectors["pipeline1:con2"].Settings, } - procCfg := processor.Config{ + procCfg1 := processor.Config{ Settings: pipeline1.Processors["pipeline1:proc1"].Settings, + Workers: pipeline1.Processors["pipeline1:proc1"].Workers, + } + procCfg2 := processor.Config{ + Settings: pipeline1.Connectors["pipeline1:con2"].Processors["pipeline1:con2:proc1con"].Settings, + Workers: pipeline1.Connectors["pipeline1:con2"].Processors["pipeline1:con2:proc1con"].Workers, } procParentPipeline := processor.Parent{ ID: pipeline1.Name, @@ -244,11 +256,11 @@ func TestProvision_Rollback(t *testing.T) { pipelineService.EXPECT().AddConnector(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:con2") pipelineService.EXPECT().RemoveConnector(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:con2") - procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con", "js", procParentConn, procCfg, processor.ProvisionTypeConfig) + procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con", "js", procParentConn, procCfg2, processor.ProvisionTypeConfig) procService.EXPECT().Delete(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con") connService.EXPECT().AddProcessor(gomock.Not(gomock.Nil()), "pipeline1:con2", "pipeline1:con2:proc1con") connService.EXPECT().RemoveProcessor(gomock.Not(gomock.Nil()), "pipeline1:con2", "pipeline1:con2:proc1con") - procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:proc1", "js", procParentPipeline, procCfg, processor.ProvisionTypeConfig) + procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:proc1", "js", procParentPipeline, procCfg1, processor.ProvisionTypeConfig) procService.EXPECT().Delete(gomock.Not(gomock.Nil()), "pipeline1:proc1") pipelineService.EXPECT().AddProcessor(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:proc1") pipelineService.EXPECT().RemoveProcessor(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:proc1") @@ -403,8 +415,13 @@ func TestProvision_ExistingPipeline(t *testing.T) { Name: pipeline1.Connectors["pipeline1:con2"].Name, Settings: pipeline1.Connectors["pipeline1:con2"].Settings, } - procCfg := processor.Config{ + procCfg1 := processor.Config{ Settings: pipeline1.Processors["pipeline1:proc1"].Settings, + Workers: pipeline1.Processors["pipeline1:proc1"].Workers, + } + procCfg2 := processor.Config{ + Settings: pipeline1.Connectors["pipeline1:con2"].Processors["pipeline1:con2:proc1con"].Settings, + Workers: pipeline1.Connectors["pipeline1:con2"].Processors["pipeline1:con2:proc1con"].Workers, } procParentPipeline := processor.Parent{ ID: pipeline1.Name, @@ -417,13 +434,13 @@ func TestProvision_ExistingPipeline(t *testing.T) { proc1Instance := &processor.Instance{ ID: "pipeline1:proc1", Parent: procParentPipeline, - Config: procCfg, + Config: procCfg1, Type: "js", } proc1conInstance := &processor.Instance{ ID: "pipeline1:con2:proc1con", Parent: procParentConn, - Config: procCfg, + Config: procCfg2, Type: "js", } source := &connector.Instance{ @@ -482,9 +499,9 @@ func TestProvision_ExistingPipeline(t *testing.T) { connService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2", connector.TypeDestination, pipeline1.Connectors["pipeline1:con2"].Plugin, pipeline1.Name, cfg2, connector.ProvisionTypeConfig).Return(destination, nil) pipelineService.EXPECT().AddConnector(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:con2") - procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con", "js", procParentConn, procCfg, processor.ProvisionTypeConfig) + procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:con2:proc1con", "js", procParentConn, procCfg2, processor.ProvisionTypeConfig) connService.EXPECT().AddProcessor(gomock.Not(gomock.Nil()), "pipeline1:con2", "pipeline1:con2:proc1con") - procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:proc1", "js", procParentPipeline, procCfg, processor.ProvisionTypeConfig) + procService.EXPECT().Create(gomock.Not(gomock.Nil()), "pipeline1:proc1", "js", procParentPipeline, procCfg1, processor.ProvisionTypeConfig) pipelineService.EXPECT().AddProcessor(gomock.Not(gomock.Nil()), pipeline1.Name, "pipeline1:proc1") pipelineService.EXPECT().Start(gomock.Not(gomock.Nil()), connService, procService, plugService, pipeline1.Name) @@ -741,7 +758,9 @@ func TestProvision_IntegrationTestServices(t *testing.T) { ID: pipeline1.Name, Type: processor.ParentTypePipeline, }, - Config: processor.Config{}, + Config: processor.Config{ + Workers: 1, + }, }, { ID: "pipeline1:con2:con2proc1", @@ -751,7 +770,9 @@ func TestProvision_IntegrationTestServices(t *testing.T) { ID: "pipeline1:con2", Type: processor.ParentTypeConnector, }, - Config: processor.Config{}, + Config: processor.Config{ + Workers: 1, + }, }, } diff --git a/pkg/provisioning/test/pipelines1/provision1-pipeline-with-processors.yml b/pkg/provisioning/test/pipelines1/provision1-pipeline-with-processors.yml index b6ee8af64..d9e27c22c 100644 --- a/pkg/provisioning/test/pipelines1/provision1-pipeline-with-processors.yml +++ b/pkg/provisioning/test/pipelines1/provision1-pipeline-with-processors.yml @@ -20,6 +20,7 @@ pipelines: processors: proc1con: type: js + workers: 10 settings: additionalProp1: string processors: diff --git a/pkg/provisioning/validate.go b/pkg/provisioning/validate.go index a60a7e2f3..0134d463e 100644 --- a/pkg/provisioning/validate.go +++ b/pkg/provisioning/validate.go @@ -71,7 +71,10 @@ func validateProcessorsConfig(mp map[string]ProcessorConfig) error { var err error for k, cfg := range mp { if cfg.Type == "" { - err = multierror.Append(err, cerrors.Errorf("processors %q: \"type\" is mandatory: %w", k, ErrMandatoryField)) + err = multierror.Append(err, cerrors.Errorf("processor %q: \"type\" is mandatory: %w", k, ErrMandatoryField)) + } + if cfg.Workers < 0 { + err = multierror.Append(err, cerrors.Errorf("processor %q: \"workers\" can't be negative: %w", k, ErrInvalidField)) } mp[k] = cfg } diff --git a/pkg/provisioning/validate_test.go b/pkg/provisioning/validate_test.go index d570792ad..15b7a113a 100644 --- a/pkg/provisioning/validate_test.go +++ b/pkg/provisioning/validate_test.go @@ -100,7 +100,7 @@ func TestValidator_InvalidFields(t *testing.T) { name string config PipelineConfig }{{ - name: "processor type is invalid", + name: "connector type is invalid", config: PipelineConfig{ Status: "running", Name: "pipeline1", @@ -137,6 +137,21 @@ func TestValidator_InvalidFields(t *testing.T) { }, }, }, + }, { + name: "processor workers is negative", + config: PipelineConfig{ + Status: "running", + Name: "pipeline1", + Description: "desc1", + Processors: map[string]ProcessorConfig{ + "proc1": { + Type: "js", + Settings: map[string]string{}, + // invalid field + Workers: -1, + }, + }, + }, }} for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { diff --git a/pkg/web/api/fromproto/processor.go b/pkg/web/api/fromproto/processor.go index 0cacb386c..2e6acb5c7 100644 --- a/pkg/web/api/fromproto/processor.go +++ b/pkg/web/api/fromproto/processor.go @@ -29,6 +29,7 @@ func _() { func ProcessorConfig(in *apiv1.Processor_Config) processor.Config { return processor.Config{ Settings: in.Settings, + Workers: int(in.Workers), } } diff --git a/pkg/web/api/toproto/processor.go b/pkg/web/api/toproto/processor.go index a2919995a..e06a3841f 100644 --- a/pkg/web/api/toproto/processor.go +++ b/pkg/web/api/toproto/processor.go @@ -41,6 +41,7 @@ func Processor(in *processor.Instance) *apiv1.Processor { func ProcessorConfig(in processor.Config) *apiv1.Processor_Config { return &apiv1.Processor_Config{ Settings: in.Settings, + Workers: int32(in.Workers), } } diff --git a/proto/api/v1/api.proto b/proto/api/v1/api.proto index ad5c933ba..82ea24ac1 100644 --- a/proto/api/v1/api.proto +++ b/proto/api/v1/api.proto @@ -119,6 +119,7 @@ message Processor { } message Config { map settings = 1; + int32 workers = 2; } string id = 1 [(google.api.field_behavior) = OUTPUT_ONLY]; diff --git a/proto/gen/api/v1/api.pb.go b/proto/gen/api/v1/api.pb.go index 0cda842c3..e08e2a971 100644 --- a/proto/gen/api/v1/api.pb.go +++ b/proto/gen/api/v1/api.pb.go @@ -3781,6 +3781,7 @@ type Processor_Config struct { unknownFields protoimpl.UnknownFields Settings map[string]string `protobuf:"bytes,1,rep,name=settings,proto3" json:"settings,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Workers int32 `protobuf:"varint,2,opt,name=workers,proto3" json:"workers,omitempty"` } func (x *Processor_Config) Reset() { @@ -3822,6 +3823,13 @@ func (x *Processor_Config) GetSettings() map[string]string { return nil } +func (x *Processor_Config) GetWorkers() int32 { + if x != nil { + return x.Workers + } + return 0 +} + type PluginSpecifications_Parameter struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4076,7 +4084,7 @@ var file_api_v1_api_proto_rawDesc = []byte{ 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44, 0x45, 0x53, 0x54, 0x49, 0x4e, 0x41, 0x54, 0x49, - 0x4f, 0x4e, 0x10, 0x02, 0x42, 0x07, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xc4, 0x04, + 0x4f, 0x4e, 0x10, 0x02, 0x42, 0x07, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xde, 0x04, 0x0a, 0x09, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x13, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x03, 0xe0, 0x41, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, @@ -4103,590 +4111,568 @@ var file_api_v1_api_proto_rawDesc = []byte{ 0x0a, 0x10, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x4f, 0x52, 0x10, 0x01, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x59, 0x50, 0x45, - 0x5f, 0x50, 0x49, 0x50, 0x45, 0x4c, 0x49, 0x4e, 0x45, 0x10, 0x02, 0x1a, 0x89, 0x01, 0x0a, 0x06, + 0x5f, 0x50, 0x49, 0x50, 0x45, 0x4c, 0x49, 0x4e, 0x45, 0x10, 0x02, 0x1a, 0xa3, 0x01, 0x0a, 0x06, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x42, 0x0a, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x1a, 0x3b, 0x0a, 0x0d, 0x53, 0x65, - 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x52, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x22, 0x2a, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x22, 0x47, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x09, 0x70, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x09, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x22, 0x48, 0x0a, 0x15, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x2f, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x22, 0x46, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, - 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x24, 0x0a, 0x12, 0x47, - 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, - 0x64, 0x22, 0x43, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x58, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, - 0x2f, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x22, 0x46, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, - 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x27, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, - 0x64, 0x22, 0x18, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x26, 0x0a, 0x14, 0x53, - 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x02, 0x69, 0x64, 0x22, 0x17, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3b, 0x0a, 0x13, - 0x53, 0x74, 0x6f, 0x70, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x53, 0x74, 0x6f, - 0x70, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x1f, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, - 0x69, 0x64, 0x22, 0x38, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x03, 0x64, 0x6c, 0x71, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x14, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x2e, 0x44, 0x4c, 0x51, 0x52, 0x03, 0x64, 0x6c, 0x71, 0x22, 0x4a, 0x0a, 0x10, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x12, 0x26, 0x0a, 0x03, 0x64, 0x6c, 0x71, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, - 0x44, 0x4c, 0x51, 0x52, 0x03, 0x64, 0x6c, 0x71, 0x22, 0x3b, 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, - 0x03, 0x64, 0x6c, 0x71, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x44, 0x4c, 0x51, - 0x52, 0x03, 0x64, 0x6c, 0x71, 0x22, 0x27, 0x0a, 0x15, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x50, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, - 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x46, - 0x0a, 0x16, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x45, 0x0a, 0x15, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x46, 0x0a, - 0x16, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, + 0x52, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x73, 0x1a, 0x3b, 0x0a, 0x0d, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x2a, + 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x47, 0x0a, 0x15, 0x4c, 0x69, + 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x09, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x09, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x73, 0x22, 0x48, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2f, 0x0a, 0x06, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x46, 0x0a, + 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0xaf, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, - 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x6c, - 0x75, 0x67, 0x69, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, - 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x22, 0x90, 0x01, 0x0a, 0x18, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, - 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x6c, - 0x75, 0x67, 0x69, 0x6e, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x1b, 0x0a, 0x19, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x38, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x49, 0x64, 0x22, 0x4b, 0x0a, - 0x16, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x0a, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x29, 0x0a, 0x17, 0x49, 0x6e, - 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x46, 0x0a, 0x18, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x6f, 0x70, 0x65, 0x6e, 0x63, 0x64, 0x63, 0x2e, 0x76, 0x31, 0x2e, 0x52, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, 0x25, 0x0a, - 0x13, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x64, 0x22, 0x47, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x24, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x43, 0x0a, 0x13, 0x47, + 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x22, 0x58, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2f, 0x0a, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x46, 0x0a, 0x16, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x22, 0x27, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x26, 0x0a, 0x14, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x17, 0x0a, + 0x15, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3b, 0x0a, 0x13, 0x53, 0x74, 0x6f, 0x70, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, + 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, + 0x72, 0x63, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x53, 0x74, 0x6f, 0x70, 0x50, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1f, 0x0a, 0x0d, 0x47, + 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x38, 0x0a, 0x0e, + 0x47, 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, + 0x0a, 0x03, 0x64, 0x6c, 0x71, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x44, 0x4c, + 0x51, 0x52, 0x03, 0x64, 0x6c, 0x71, 0x22, 0x4a, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x26, 0x0a, 0x03, 0x64, 0x6c, + 0x71, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x44, 0x4c, 0x51, 0x52, 0x03, 0x64, + 0x6c, 0x71, 0x22, 0x3b, 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, 0x51, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x03, 0x64, 0x6c, 0x71, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x44, 0x4c, 0x51, 0x52, 0x03, 0x64, 0x6c, 0x71, 0x22, + 0x27, 0x0a, 0x15, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x46, 0x0a, 0x16, 0x45, 0x78, 0x70, 0x6f, + 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x22, 0x45, 0x0a, 0x15, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x46, 0x0a, 0x16, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, + 0xaf, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x12, 0x1f, + 0x0a, 0x0b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x49, 0x64, 0x12, + 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x5a, 0x0a, - 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x28, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, + 0x6f, 0x72, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x90, 0x01, + 0x0a, 0x18, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x12, 0x30, + 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x22, 0x1b, 0x0a, 0x19, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x38, 0x0a, + 0x15, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x49, 0x64, 0x22, 0x4b, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x31, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x73, 0x22, 0x29, 0x0a, 0x17, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, - 0x19, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x0a, 0x15, 0x4c, 0x69, - 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, - 0x64, 0x73, 0x22, 0x4b, 0x0a, 0x16, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x0a, - 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x22, - 0x2b, 0x0a, 0x19, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x48, 0x0a, 0x1a, - 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, - 0x49, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, - 0x63, 0x6f, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x6f, 0x70, 0x65, - 0x6e, 0x63, 0x64, 0x63, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x06, - 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, 0x2c, 0x0a, 0x1a, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, - 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x64, 0x22, 0x49, 0x0a, 0x1b, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, - 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x6f, 0x70, 0x65, 0x6e, 0x63, 0x64, 0x63, 0x2e, 0x76, 0x31, - 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, - 0x90, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x30, - 0x0a, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, - 0x72, 0x2e, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, - 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, - 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, - 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, 0x25, - 0x0a, 0x13, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x47, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, - 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, - 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, 0x5a, - 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, - 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x09, 0x70, 0x72, 0x6f, - 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, 0x28, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x22, 0x19, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x47, - 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x33, 0x0a, - 0x0f, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x20, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, - 0x66, 0x6f, 0x22, 0x44, 0x0a, 0x04, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x6f, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x72, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x61, 0x72, 0x63, 0x68, 0x22, 0x28, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, - 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, - 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x22, 0x4d, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x70, 0x6c, 0x75, - 0x67, 0x69, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, - 0x73, 0x22, 0x8c, 0x09, 0x0a, 0x14, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, - 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, - 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x12, 0x62, 0x0a, 0x12, - 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, - 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x64, - 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, - 0x12, 0x53, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, - 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x61, 0x72, 0x61, - 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, - 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0xe1, 0x04, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, - 0x74, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, - 0x3f, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2b, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, - 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, - 0x6d, 0x65, 0x74, 0x65, 0x72, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, - 0x12, 0x53, 0x0a, 0x0b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, - 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x2e, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x83, 0x02, 0x0a, 0x0a, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, - 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x92, 0x01, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x14, 0x0a, 0x10, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, - 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, - 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x54, 0x59, 0x50, 0x45, - 0x5f, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x02, 0x12, - 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, - 0x4e, 0x10, 0x03, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x43, 0x4c, - 0x55, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x5f, - 0x45, 0x58, 0x43, 0x4c, 0x55, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, - 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x47, 0x45, 0x58, 0x10, 0x06, 0x22, 0x7c, 0x0a, 0x04, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, - 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, - 0x45, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x59, - 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, - 0x5f, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x59, 0x50, 0x45, - 0x5f, 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x59, 0x50, 0x45, 0x5f, - 0x46, 0x49, 0x4c, 0x45, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44, - 0x55, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x1a, 0x6c, 0x0a, 0x16, 0x44, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, + 0x46, 0x0a, 0x18, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x6f, 0x70, + 0x65, 0x6e, 0x63, 0x64, 0x63, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, + 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, 0x25, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x47, + 0x0a, 0x14, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x63, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x5a, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, + 0x64, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, + 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, + 0x28, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x19, 0x0a, 0x17, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, + 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x4b, 0x0a, 0x16, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x0a, 0x70, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x22, 0x2b, 0x0a, 0x19, 0x49, 0x6e, 0x73, + 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x48, 0x0a, 0x1a, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, + 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x6f, 0x70, 0x65, 0x6e, 0x63, 0x64, 0x63, 0x2e, 0x76, + 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, + 0x22, 0x2c, 0x0a, 0x1a, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x49, + 0x0a, 0x1b, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, + 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x6f, 0x70, 0x65, 0x6e, 0x63, 0x64, 0x63, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x52, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, 0x90, 0x01, 0x0a, 0x16, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x2e, 0x50, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x52, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x2e, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x09, 0x70, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, 0x25, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x50, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, + 0x47, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x09, 0x70, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, 0x5a, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, + 0x69, 0x64, 0x12, 0x30, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x22, 0x4a, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x2f, 0x0a, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, + 0x22, 0x28, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x19, 0x0a, 0x17, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x33, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x20, 0x0a, 0x04, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0x44, 0x0a, 0x04, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, + 0x0a, 0x02, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x6f, 0x73, 0x12, 0x12, + 0x0a, 0x04, 0x61, 0x72, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x61, 0x72, + 0x63, 0x68, 0x22, 0x28, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x4d, 0x0a, 0x13, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x36, 0x0a, 0x07, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x67, 0x0a, 0x11, 0x53, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3c, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, - 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, - 0x6d, 0x65, 0x74, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x32, 0xe6, 0x15, 0x0a, 0x0f, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x12, 0x6e, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, - 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x20, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1a, 0x62, 0x09, 0x70, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, 0x0d, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x73, 0x12, 0xe7, 0x02, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x95, 0x02, 0x92, 0x41, 0xef, 0x01, 0x4a, 0x77, 0x0a, - 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, - 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, + 0x6e, 0x73, 0x52, 0x07, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x22, 0x8c, 0x09, 0x0a, 0x14, + 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, + 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, + 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, + 0x0a, 0x06, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, + 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x12, 0x62, 0x0a, 0x12, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, + 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x2e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, + 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x53, 0x0a, 0x0d, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x2e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, + 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, + 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, + 0xe1, 0x04, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x12, 0x20, 0x0a, + 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x18, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x3f, 0x0a, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x2e, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, 0x0b, 0x76, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x31, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, + 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x0b, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, + 0x83, 0x02, 0x0a, 0x0a, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4a, + 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, + 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, + 0x65, 0x74, 0x65, 0x72, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x22, 0x92, 0x01, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x11, 0x0a, 0x0d, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, + 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x47, 0x52, 0x45, 0x41, 0x54, + 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, + 0x45, 0x5f, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x03, 0x12, 0x12, 0x0a, + 0x0e, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x43, 0x4c, 0x55, 0x53, 0x49, 0x4f, 0x4e, 0x10, + 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x58, 0x43, 0x4c, 0x55, 0x53, + 0x49, 0x4f, 0x4e, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x45, + 0x47, 0x45, 0x58, 0x10, 0x06, 0x22, 0x7c, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, + 0x10, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x52, 0x49, + 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x54, + 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x4c, 0x4f, 0x41, 0x54, + 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x42, 0x4f, 0x4f, 0x4c, 0x10, + 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x49, 0x4c, 0x45, 0x10, 0x05, + 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44, 0x55, 0x52, 0x41, 0x54, 0x49, 0x4f, + 0x4e, 0x10, 0x06, 0x1a, 0x6c, 0x0a, 0x16, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x3c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, + 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x67, 0x0a, 0x11, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xe6, 0x15, 0x0a, 0x0f, 0x50, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x6e, + 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, + 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x20, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x1a, 0x62, 0x09, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, + 0x0d, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, 0xe7, + 0x02, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x95, 0x02, 0x92, 0x41, 0xef, 0x01, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, + 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, + 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, + 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, + 0x4a, 0x74, 0x0a, 0x03, 0x34, 0x30, 0x39, 0x12, 0x6d, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x53, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x3f, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, + 0x20, 0x36, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, + 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x20, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, - 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x74, 0x0a, 0x03, 0x34, 0x30, 0x39, 0x12, 0x6d, 0x12, - 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x53, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x3f, 0x7b, 0x20, 0x22, - 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x36, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x20, 0x65, 0x78, - 0x69, 0x73, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, - 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, - 0x02, 0x1c, 0x3a, 0x01, 0x2a, 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, - 0x0d, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, 0xea, - 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1a, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa1, 0x01, 0x92, 0x41, 0x7a, 0x4a, 0x78, 0x0a, - 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, + 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x3a, 0x01, 0x2a, 0x62, + 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x0d, 0x2f, 0x76, 0x31, 0x2f, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, 0xea, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0xa1, 0x01, 0x92, 0x41, 0x7a, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, + 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, + 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, + 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, + 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x12, 0x12, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, + 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe6, 0x03, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x94, 0x03, 0x92, 0x41, 0xe9, 0x02, 0x4a, 0x77, + 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, + 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, + 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, + 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, - 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, 0x62, 0x08, 0x70, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x12, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe6, 0x03, 0x0a, 0x0e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1d, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x94, 0x03, - 0x92, 0x41, 0xe9, 0x02, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, - 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, - 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, - 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, - 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, - 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, + 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, + 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, + 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, + 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, + 0x7d, 0x4a, 0x74, 0x0a, 0x03, 0x34, 0x30, 0x39, 0x12, 0x6d, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x53, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x3f, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, + 0x3a, 0x20, 0x36, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, + 0x22, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x20, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, - 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x74, 0x0a, 0x03, 0x34, 0x30, 0x39, 0x12, 0x6d, + 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x21, 0x3a, 0x01, 0x2a, + 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x1a, 0x12, 0x2f, 0x76, 0x31, 0x2f, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe5, + 0x02, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x93, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, - 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x53, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x3f, 0x7b, 0x20, - 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x36, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x20, 0x65, - 0x78, 0x69, 0x73, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, - 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x21, 0x3a, 0x01, 0x2a, 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x1a, 0x12, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, - 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe5, 0x02, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x93, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, - 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, - 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, - 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, - 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, - 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, - 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, - 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x14, 0x2a, 0x12, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe8, 0x02, 0x0a, - 0x0d, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1c, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x99, 0x02, 0x92, 0x41, - 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, + 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, - 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, - 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, - 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, - 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, - 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1a, 0x22, 0x18, 0x2f, - 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, - 0x7d, 0x2f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0xe7, 0x02, 0x0a, 0x0c, 0x53, 0x74, 0x6f, 0x70, - 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, - 0x74, 0x6f, 0x70, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, - 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, - 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, - 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, - 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, - 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, - 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, - 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, - 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, - 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, - 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x3a, 0x01, 0x2a, 0x22, 0x17, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x74, 0x6f, - 0x70, 0x12, 0x6a, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x44, 0x4c, 0x51, 0x12, 0x15, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x44, - 0x4c, 0x51, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, - 0x02, 0x2b, 0x62, 0x03, 0x64, 0x6c, 0x71, 0x12, 0x24, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x64, 0x65, 0x61, 0x64, - 0x2d, 0x6c, 0x65, 0x74, 0x74, 0x65, 0x72, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x12, 0x78, 0x0a, - 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, 0x51, 0x12, 0x18, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x30, 0x3a, 0x03, 0x64, 0x6c, 0x71, 0x62, 0x03, 0x64, 0x6c, - 0x71, 0x1a, 0x24, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, - 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x64, 0x65, 0x61, 0x64, 0x2d, 0x6c, 0x65, 0x74, 0x74, 0x65, - 0x72, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x12, 0x7c, 0x0a, 0x0e, 0x45, 0x78, 0x70, 0x6f, 0x72, - 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x76, 0x31, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x25, - 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x19, 0x2f, 0x76, 0x31, 0x2f, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x65, - 0x78, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, - 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x30, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2a, 0x3a, - 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x22, 0x14, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x73, 0x2f, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x32, 0x81, 0x0f, 0x0a, 0x10, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x73, - 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, - 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x22, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x62, 0x0a, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x73, 0x12, 0x0e, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x73, 0x12, 0x84, 0x01, 0x0a, 0x10, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2b, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x25, 0x62, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x1b, 0x2f, 0x76, 0x31, - 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, - 0x2f, 0x69, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x30, 0x01, 0x12, 0xef, 0x01, 0x0a, 0x0c, 0x47, - 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1b, 0x2e, 0x61, 0x70, - 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, - 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa3, 0x01, 0x92, 0x41, 0x7a, 0x4a, 0x78, 0x0a, 0x03, - 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, + 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, + 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, + 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, + 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, + 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x14, 0x2a, 0x12, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe8, 0x02, 0x0a, 0x0d, 0x53, 0x74, 0x61, 0x72, 0x74, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x99, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, + 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, - 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, - 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, - 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x20, 0x62, 0x09, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xf5, 0x01, 0x0a, - 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, - 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0xa0, 0x01, 0x92, 0x41, 0x79, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, + 0x6e, 0x12, 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, + 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, - 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, - 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, - 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, - 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, - 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x6f, 0x72, 0x22, 0x0e, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x73, 0x12, 0xa2, 0x03, 0x0a, 0x11, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x20, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0xc7, 0x02, 0x92, 0x41, 0xa1, 0x02, 0x4a, 0x9d, 0x01, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x95, - 0x01, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, - 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x7b, 0x0a, 0x10, 0x61, 0x70, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x67, 0x7b, - 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x3a, 0x20, 0x60, 0x61, 0x77, 0x73, 0x2e, 0x61, - 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x49, 0x64, 0x60, 0x20, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x20, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x20, 0x6d, 0x75, 0x73, 0x74, 0x20, 0x62, 0x65, - 0x20, 0x73, 0x65, 0x74, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, - 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x7f, 0x0a, 0x03, 0x35, 0x30, 0x30, 0x12, 0x78, 0x12, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, + 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, + 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, + 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1a, 0x22, 0x18, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x12, 0xe7, 0x02, 0x0a, 0x0c, 0x53, 0x74, 0x6f, 0x70, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x12, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x6f, 0x70, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x02, + 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, + 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, + 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, + 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, + 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, + 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, + 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, + 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, + 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x3a, + 0x01, 0x2a, 0x22, 0x17, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x73, 0x74, 0x6f, 0x70, 0x12, 0x6a, 0x0a, 0x06, 0x47, + 0x65, 0x74, 0x44, 0x4c, 0x51, 0x12, 0x15, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, + 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2b, 0x62, 0x03, 0x64, 0x6c, + 0x71, 0x12, 0x24, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, + 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x64, 0x65, 0x61, 0x64, 0x2d, 0x6c, 0x65, 0x74, 0x74, 0x65, + 0x72, 0x2d, 0x71, 0x75, 0x65, 0x75, 0x65, 0x12, 0x78, 0x0a, 0x09, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x44, 0x4c, 0x51, 0x12, 0x18, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, 0x51, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x4c, + 0x51, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x36, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x30, 0x3a, 0x03, 0x64, 0x6c, 0x71, 0x62, 0x03, 0x64, 0x6c, 0x71, 0x1a, 0x24, 0x2f, 0x76, 0x31, + 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, + 0x64, 0x65, 0x61, 0x64, 0x2d, 0x6c, 0x65, 0x74, 0x74, 0x65, 0x72, 0x2d, 0x71, 0x75, 0x65, 0x75, + 0x65, 0x12, 0x7c, 0x0a, 0x0e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x70, 0x6f, + 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x2b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x25, 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x19, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x65, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x12, + 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x30, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x2a, 0x3a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x62, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x14, 0x2f, + 0x76, 0x31, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2f, 0x69, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x32, 0x81, 0x0f, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x73, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x22, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x1c, 0x62, 0x0a, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x0e, 0x2f, + 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x84, 0x01, + 0x0a, 0x10, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x12, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, + 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, + 0x70, 0x65, 0x63, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x25, 0x62, 0x06, 0x72, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x1b, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x69, 0x6e, 0x73, 0x70, 0x65, + 0x63, 0x74, 0x30, 0x01, 0x12, 0xef, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, + 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0xa3, 0x01, 0x92, 0x41, 0x7a, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x5e, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x4a, 0x7b, 0x20, 0x22, - 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x31, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x63, 0x6f, 0x75, 0x6c, 0x64, 0x20, 0x6e, 0x6f, 0x74, - 0x20, 0x64, 0x69, 0x73, 0x70, 0x65, 0x6e, 0x73, 0x65, 0x20, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, - 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x3a, 0x01, 0x2a, - 0x22, 0x17, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, - 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x12, 0xf5, 0x02, 0x0a, 0x0f, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1e, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, - 0x02, 0x92, 0x41, 0xf3, 0x01, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, - 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, - 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, - 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, - 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, - 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, - 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, - 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, - 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, - 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x23, 0x3a, 0x01, - 0x2a, 0x62, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x1a, 0x13, 0x2f, 0x76, - 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, - 0x7d, 0x12, 0xe9, 0x02, 0x0a, 0x0f, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x94, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, - 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, - 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, + 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, + 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, + 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, + 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x20, 0x62, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x12, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, + 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xf5, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, 0x01, 0x92, 0x41, + 0x79, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, + 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, + 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, + 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x0e, + 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0xa2, + 0x03, 0x0a, 0x11, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x12, 0x20, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc7, 0x02, 0x92, 0x41, 0xa1, 0x02, + 0x4a, 0x9d, 0x01, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x95, 0x01, 0x12, 0x16, 0x0a, 0x14, 0x1a, + 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x7b, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x67, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, + 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, + 0x20, 0x22, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x3a, 0x20, 0x60, 0x61, 0x77, 0x73, 0x2e, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, + 0x65, 0x79, 0x49, 0x64, 0x60, 0x20, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x20, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x20, 0x6d, 0x75, 0x73, 0x74, 0x20, 0x62, 0x65, 0x20, 0x73, 0x65, 0x74, 0x22, 0x2c, + 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, + 0x4a, 0x7f, 0x0a, 0x03, 0x35, 0x30, 0x30, 0x12, 0x78, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x5e, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x4a, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, + 0x20, 0x31, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, + 0x22, 0x63, 0x6f, 0x75, 0x6c, 0x64, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x64, 0x69, 0x73, 0x70, 0x65, + 0x6e, 0x73, 0x65, 0x20, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, + 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, + 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x3a, 0x01, 0x2a, 0x22, 0x17, 0x2f, 0x76, 0x31, 0x2f, + 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x65, 0x12, 0xf5, 0x02, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, 0x02, 0x92, 0x41, 0xf3, 0x01, 0x4a, + 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, + 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, + 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, + 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, @@ -4696,101 +4682,101 @@ var file_api_v1_api_proto_rawDesc = []byte{ 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, - 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x2a, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x32, 0xf9, 0x0c, - 0x0a, 0x10, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x73, 0x0a, 0x0e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x73, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x22, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x62, 0x0a, 0x70, 0x72, 0x6f, - 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x12, 0x0e, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, - 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x12, 0x8d, 0x01, 0x0a, 0x12, 0x49, 0x6e, 0x73, 0x70, - 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x12, 0x21, - 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, - 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x22, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, - 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x28, 0x62, 0x06, 0x72, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x1e, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, - 0x73, 0x73, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x69, 0x6e, 0x73, 0x70, 0x65, - 0x63, 0x74, 0x2d, 0x69, 0x6e, 0x30, 0x01, 0x12, 0x91, 0x01, 0x0a, 0x13, 0x49, 0x6e, 0x73, 0x70, - 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x12, - 0x22, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, - 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, - 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x29, - 0x62, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x1f, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, - 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x69, 0x6e, - 0x73, 0x70, 0x65, 0x63, 0x74, 0x2d, 0x6f, 0x75, 0x74, 0x30, 0x01, 0x12, 0xef, 0x01, 0x0a, 0x0c, - 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x1b, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, - 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa3, 0x01, 0x92, 0x41, 0x7a, 0x4a, 0x78, 0x0a, - 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, - 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, - 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, - 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, - 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x20, 0x62, 0x09, 0x70, - 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, - 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xf5, 0x01, - 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, - 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0xa0, 0x01, 0x92, 0x41, 0x79, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, - 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, - 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, - 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, - 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, + 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x23, 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x63, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x1a, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe9, 0x02, 0x0a, 0x0f, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x12, + 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x94, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, + 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, + 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, + 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, + 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, + 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, + 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, + 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, + 0x02, 0x15, 0x2a, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x32, 0xf9, 0x0c, 0x0a, 0x10, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x6f, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x73, 0x0a, 0x0e, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x12, 0x1d, + 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x22, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x62, 0x0a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, + 0x73, 0x12, 0x0e, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, + 0x73, 0x12, 0x8d, 0x01, 0x0a, 0x12, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x12, 0x21, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, + 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x6f, 0x72, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x6f, 0x72, 0x49, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x2e, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x28, 0x62, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, + 0x1e, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x2f, + 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x69, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x2d, 0x69, 0x6e, 0x30, + 0x01, 0x12, 0x91, 0x01, 0x0a, 0x13, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x12, 0x22, 0x2e, 0x61, 0x70, 0x69, 0x2e, + 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, + 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x50, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x4f, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x2f, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x29, 0x62, 0x06, 0x72, 0x65, 0x63, 0x6f, + 0x72, 0x64, 0x12, 0x1f, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, + 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x2f, 0x69, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x74, 0x2d, + 0x6f, 0x75, 0x74, 0x30, 0x01, 0x12, 0xef, 0x01, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, + 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0xa3, 0x01, 0x92, 0x41, 0x7a, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, + 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, + 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, + 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, - 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1e, 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x70, 0x72, 0x6f, 0x63, - 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, 0x0e, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, - 0x73, 0x73, 0x6f, 0x72, 0x73, 0x12, 0xf5, 0x02, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, - 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, - 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, - 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, 0x02, 0x92, 0x41, 0xf3, - 0x01, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, - 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, - 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, - 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, - 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x57, 0x0a, 0x10, 0x61, - 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, - 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, 0x2c, 0x20, 0x22, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, - 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x23, 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x70, - 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x1a, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, - 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe9, 0x02, - 0x0a, 0x0f, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, - 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x94, 0x02, 0x92, 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, 0x30, - 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x44, - 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x20, - 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, + 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x20, 0x62, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x6f, 0x72, 0x12, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, + 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xf5, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, 0x01, 0x92, + 0x41, 0x79, 0x4a, 0x77, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, + 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, + 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, + 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, + 0x1e, 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x22, + 0x0e, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x12, + 0xf5, 0x02, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa0, 0x02, 0x92, 0x41, 0xf3, 0x01, 0x4a, 0x77, 0x0a, 0x03, 0x34, + 0x30, 0x30, 0x12, 0x70, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x56, 0x0a, 0x10, + 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, + 0x12, 0x42, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x33, 0x2c, 0x20, 0x22, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x69, 0x6e, 0x76, 0x61, 0x6c, + 0x69, 0x64, 0x20, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, @@ -4800,49 +4786,73 @@ var file_api_v1_api_proto_rawDesc = []byte{ 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, - 0xe4, 0x93, 0x02, 0x15, 0x2a, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x6f, 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x32, 0x65, 0x0a, 0x12, 0x49, 0x6e, 0x66, - 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, - 0x4f, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x2e, 0x61, 0x70, 0x69, - 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x13, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x0d, 0x62, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x05, 0x2f, 0x69, 0x6e, 0x66, 0x6f, - 0x32, 0x75, 0x0a, 0x0d, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x12, 0x64, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, - 0x12, 0x1a, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, - 0x75, 0x67, 0x69, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x61, - 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1c, 0x82, 0xd3, 0xe4, 0x93, 0x02, - 0x16, 0x62, 0x07, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x12, 0x0b, 0x2f, 0x76, 0x31, 0x2f, - 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x42, 0xa6, 0x03, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x2e, - 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x42, 0x08, 0x41, 0x70, 0x69, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x50, 0x01, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, - 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x69, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, - 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, - 0x31, 0x3b, 0x61, 0x70, 0x69, 0x76, 0x31, 0xa2, 0x02, 0x03, 0x41, 0x58, 0x58, 0xaa, 0x02, 0x06, - 0x41, 0x70, 0x69, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x06, 0x41, 0x70, 0x69, 0x5c, 0x56, 0x31, 0xe2, - 0x02, 0x12, 0x41, 0x70, 0x69, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x07, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x56, 0x31, 0x92, 0x41, - 0x9e, 0x02, 0x12, 0xac, 0x01, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x20, 0x52, - 0x45, 0x53, 0x54, 0x20, 0x41, 0x50, 0x49, 0x22, 0x37, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x64, 0x75, - 0x69, 0x74, 0x20, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x24, 0x68, 0x74, 0x74, 0x70, - 0x73, 0x3a, 0x2f, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, - 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x69, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, - 0x2a, 0x57, 0x0a, 0x1a, 0x41, 0x70, 0x61, 0x63, 0x68, 0x65, 0x20, 0x4c, 0x69, 0x63, 0x65, 0x6e, - 0x73, 0x65, 0x20, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x20, 0x32, 0x2e, 0x30, 0x12, 0x39, - 0x68, 0x74, 0x74, 0x70, 0x73, 0x3a, 0x2f, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, - 0x6f, 0x6d, 0x2f, 0x43, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x49, 0x4f, 0x2f, 0x63, 0x6f, 0x6e, - 0x64, 0x75, 0x69, 0x74, 0x2f, 0x62, 0x6c, 0x6f, 0x62, 0x2f, 0x6d, 0x61, 0x69, 0x6e, 0x2f, 0x4c, - 0x49, 0x43, 0x45, 0x4e, 0x53, 0x45, 0x2e, 0x6d, 0x64, 0x32, 0x06, 0x76, 0x30, 0x2e, 0x31, 0x2e, - 0x30, 0x52, 0x6d, 0x0a, 0x03, 0x35, 0x30, 0x30, 0x12, 0x66, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x4c, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x38, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, - 0x3a, 0x20, 0x31, 0x33, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, - 0x20, 0x22, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, - 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0xe4, 0x93, 0x02, 0x23, 0x3a, 0x01, 0x2a, 0x62, 0x09, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x6f, 0x72, 0x1a, 0x13, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, + 0x72, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x7d, 0x12, 0xe9, 0x02, 0x0a, 0x0f, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x12, 0x1e, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x61, 0x70, + 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x94, 0x02, 0x92, + 0x41, 0xf5, 0x01, 0x4a, 0x79, 0x0a, 0x03, 0x34, 0x30, 0x30, 0x12, 0x72, 0x12, 0x16, 0x0a, 0x14, + 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x58, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, 0x12, 0x44, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, + 0x65, 0x22, 0x3a, 0x20, 0x39, 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x3a, 0x20, 0x22, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x20, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6e, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, + 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x4a, 0x78, + 0x0a, 0x03, 0x34, 0x30, 0x34, 0x12, 0x71, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0x57, 0x0a, 0x10, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, + 0x73, 0x6f, 0x6e, 0x12, 0x43, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x35, + 0x2c, 0x20, 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x66, 0x6f, 0x75, 0x6e, 0x64, + 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x2a, 0x13, + 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x6f, 0x72, 0x73, 0x2f, 0x7b, + 0x69, 0x64, 0x7d, 0x32, 0x65, 0x0a, 0x12, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4f, 0x0a, 0x07, 0x47, 0x65, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x61, + 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x13, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x0d, 0x62, 0x04, 0x69, + 0x6e, 0x66, 0x6f, 0x12, 0x05, 0x2f, 0x69, 0x6e, 0x66, 0x6f, 0x32, 0x75, 0x0a, 0x0d, 0x50, 0x6c, + 0x75, 0x67, 0x69, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x64, 0x0a, 0x0b, 0x4c, + 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x12, 0x1a, 0x2e, 0x61, 0x70, 0x69, + 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x1c, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x16, 0x62, 0x07, 0x70, 0x6c, 0x75, + 0x67, 0x69, 0x6e, 0x73, 0x12, 0x0b, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, + 0x73, 0x42, 0xa6, 0x03, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x2e, 0x61, 0x70, 0x69, 0x2e, 0x76, 0x31, + 0x42, 0x08, 0x41, 0x70, 0x69, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x33, 0x67, 0x69, + 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, + 0x69, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x3b, 0x61, 0x70, 0x69, 0x76, + 0x31, 0xa2, 0x02, 0x03, 0x41, 0x58, 0x58, 0xaa, 0x02, 0x06, 0x41, 0x70, 0x69, 0x2e, 0x56, 0x31, + 0xca, 0x02, 0x06, 0x41, 0x70, 0x69, 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x12, 0x41, 0x70, 0x69, 0x5c, + 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, + 0x07, 0x41, 0x70, 0x69, 0x3a, 0x3a, 0x56, 0x31, 0x92, 0x41, 0x9e, 0x02, 0x12, 0xac, 0x01, 0x0a, + 0x10, 0x43, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x20, 0x52, 0x45, 0x53, 0x54, 0x20, 0x41, 0x50, + 0x49, 0x22, 0x37, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x20, 0x70, 0x72, 0x6f, + 0x6a, 0x65, 0x63, 0x74, 0x12, 0x24, 0x68, 0x74, 0x74, 0x70, 0x73, 0x3a, 0x2f, 0x2f, 0x67, 0x69, + 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, + 0x69, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x2a, 0x57, 0x0a, 0x1a, 0x41, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x20, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x20, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x20, 0x32, 0x2e, 0x30, 0x12, 0x39, 0x68, 0x74, 0x74, 0x70, 0x73, 0x3a, + 0x2f, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x43, 0x6f, 0x6e, + 0x64, 0x75, 0x69, 0x74, 0x49, 0x4f, 0x2f, 0x63, 0x6f, 0x6e, 0x64, 0x75, 0x69, 0x74, 0x2f, 0x62, + 0x6c, 0x6f, 0x62, 0x2f, 0x6d, 0x61, 0x69, 0x6e, 0x2f, 0x4c, 0x49, 0x43, 0x45, 0x4e, 0x53, 0x45, + 0x2e, 0x6d, 0x64, 0x32, 0x06, 0x76, 0x30, 0x2e, 0x31, 0x2e, 0x30, 0x52, 0x6d, 0x0a, 0x03, 0x35, + 0x30, 0x30, 0x12, 0x66, 0x12, 0x16, 0x0a, 0x14, 0x1a, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x4c, 0x0a, 0x10, + 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x6a, 0x73, 0x6f, 0x6e, + 0x12, 0x38, 0x7b, 0x20, 0x22, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x3a, 0x20, 0x31, 0x33, 0x2c, 0x20, + 0x22, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x3a, 0x20, 0x22, 0x73, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x20, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x2c, 0x20, 0x22, 0x64, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x22, 0x3a, 0x20, 0x5b, 0x5d, 0x20, 0x7d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, } var ( diff --git a/proto/gen/api/v1/api.swagger.json b/proto/gen/api/v1/api.swagger.json index 7fc14921e..c4f9f9b98 100644 --- a/proto/gen/api/v1/api.swagger.json +++ b/proto/gen/api/v1/api.swagger.json @@ -2071,6 +2071,10 @@ "additionalProperties": { "type": "string" } + }, + "workers": { + "type": "integer", + "format": "int32" } } },