From 18ab665353939df4bd9a4ef0e6fdc4e506c7195f Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Wed, 17 May 2023 21:39:51 +0200 Subject: [PATCH 1/7] Turning off defer secondary keys since it needs to be more nuanced for fk-related tables. MoveTables for parent and child works with no load Signed-off-by: Rohit Nayak Add failing e2e test Signed-off-by: Rohit Nayak Add tablestreamer to stream entire table w/unit test Signed-off-by: Rohit Nayak Implement copy phase in foreign key mode Signed-off-by: Rohit Nayak Add load testing for e2e test Signed-off-by: Rohit Nayak Also send RowEvent flags in vstreamer and use it in vplayer to set foreign_key_checks setting correctly Signed-off-by: Rohit Nayak Cleanup Signed-off-by: Rohit Nayak Add SwitchTraffic to fk e2e test with continuing load Signed-off-by: Rohit Nayak WIP Signed-off-by: Rohit Nayak Fix two unit tests Signed-off-by: Rohit Nayak Added comment Signed-off-by: Rohit Nayak Fix vstreamer tests. Some refactoring/comments Signed-off-by: Rohit Nayak Fix vreplication unit tests. Some refactor Signed-off-by: Rohit Nayak Add ci workflow for FK Mode. Currently in a new shard for visibility Signed-off-by: Rohit Nayak Return copy all error correctly. Do not retry such an error. Signed-off-by: Rohit Nayak Save lastpk in copy_state Signed-off-by: Rohit Nayak Don't allow restarting FK Mode workflows which have errored in copy phase. Fix saving of RowsCopied into vrep table. Signed-off-by: Rohit Nayak Removed forced panic added for specific testing Signed-off-by: Rohit Nayak Add the foreign-key-mode to the vtctldclient code. Signed-off-by: Rohit Nayak Add abstraction to switch randomly between vtctl and vtctld for MoveTables with foreign key mode Signed-off-by: Rohit Nayak Address review comments Signed-off-by: Rohit Nayak Address review comments Signed-off-by: Rohit Nayak Fix tests which broke due to refactoring Signed-off-by: Rohit Nayak Check for incompatible options for foreign key mode Signed-off-by: Rohit Nayak Extract checkForeignKeyModeOptions Signed-off-by: Rohit Nayak Self-review Signed-off-by: Rohit Nayak Fix regressions Signed-off-by: Rohit Nayak Fix tests for vtctld which uses --all-tables and not --all Signed-off-by: Rohit Nayak Change terminology from foreign key mode for the VReplication MoveTables flag, to atomic copy Signed-off-by: Rohit Nayak Move FK test to cellalias shard Signed-off-by: Rohit Nayak --- Makefile | 1 + go/cmd/vtctldclient/command/movetables.go | 26 + go/cmd/vtctldclient/command/workflows.go | 40 + go/test/endtoend/vreplication/cluster_test.go | 3 +- .../endtoend/vreplication/fk_config_test.go | 65 + go/test/endtoend/vreplication/fk_test.go | 256 ++ .../vreplication/movetables_buffering_test.go | 2 +- .../partial_movetables_seq_test.go | 12 +- .../vreplication/partial_movetables_test.go | 12 +- .../resharding_workflows_v2_test.go | 35 +- .../endtoend/vreplication/wrappers_test.go | 191 + go/vt/proto/binlogdata/binlogdata.pb.go | 827 ++-- .../proto/binlogdata/binlogdata_vtproto.pb.go | 701 ++++ go/vt/proto/queryservice/queryservice.pb.go | 160 +- .../queryservice/queryservice_grpc.pb.go | 69 +- go/vt/proto/vtctldata/vtctldata.pb.go | 3585 +++++++++-------- go/vt/proto/vtctldata/vtctldata_vtproto.pb.go | 70 + go/vt/vtcombo/tablet_map.go | 10 + go/vt/vtctl/vtctl.go | 20 + go/vt/vtctl/workflow/materializer.go | 23 +- go/vt/vtctl/workflow/server.go | 1 + go/vt/vtgate/endtoend/vstream_test.go | 2 + go/vt/vttablet/endtoend/vstreamer_test.go | 4 + go/vt/vttablet/grpcqueryservice/server.go | 10 + go/vt/vttablet/grpctabletconn/conn.go | 40 + go/vt/vttablet/queryservice/queryservice.go | 4 + go/vt/vttablet/queryservice/wrapped.go | 7 + go/vt/vttablet/sandboxconn/sandboxconn.go | 5 + .../tabletconntest/fakequeryservice.go | 5 + .../tabletmanager/vreplication/controller.go | 18 +- .../vreplication/external_connector.go | 12 + .../vreplication/framework_test.go | 33 +- .../tabletmanager/vreplication/vcopier.go | 21 +- .../vreplication/vcopier_copy_all.go | 297 ++ .../vreplication/vcopier_test.go | 16 +- .../tabletmanager/vreplication/vplayer.go | 28 + .../vreplication/vplayer_flaky_test.go | 56 + .../tabletmanager/vreplication/vreplicator.go | 38 +- go/vt/vttablet/tabletserver/tabletserver.go | 8 + .../vttablet/tabletserver/vstreamer/engine.go | 47 +- .../tabletserver/vstreamer/main_flaky_test.go | 1 + .../tabletserver/vstreamer/rowstreamer.go | 79 +- .../tabletserver/vstreamer/snapshot_conn.go | 40 + .../tabletserver/vstreamer/tablestreamer.go | 176 + .../vstreamer/tablestreamer_test.go | 78 + .../vstreamer/uvstreamer_flaky_test.go | 3 + .../tabletserver/vstreamer/vstreamer.go | 2 +- .../vstreamer/vstreamer_flaky_test.go | 49 + go/vt/wrangler/materializer.go | 49 +- go/vt/wrangler/vexec.go | 46 +- go/vt/wrangler/workflow.go | 4 +- proto/binlogdata.proto | 20 + proto/queryservice.proto | 3 + proto/vtctldata.proto | 3 + test/config.json | 9 + web/vtadmin/src/proto/vtadmin.d.ts | 257 +- web/vtadmin/src/proto/vtadmin.js | 735 ++++ 57 files changed, 6032 insertions(+), 2282 deletions(-) create mode 100644 go/test/endtoend/vreplication/fk_config_test.go create mode 100644 go/test/endtoend/vreplication/fk_test.go create mode 100644 go/test/endtoend/vreplication/wrappers_test.go create mode 100644 go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go create mode 100644 go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go create mode 100644 go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go diff --git a/Makefile b/Makefile index e8761f127ed..9c7fbe0c304 100644 --- a/Makefile +++ b/Makefile @@ -267,6 +267,7 @@ $(PROTO_GO_OUTS): minimaltools install_protoc-gen-go proto/*.proto --go-vtproto_opt=features=marshal+unmarshal+size+pool+clone \ --go-vtproto_opt=pool=vitess.io/vitess/go/vt/proto/query.Row \ --go-vtproto_opt=pool=vitess.io/vitess/go/vt/proto/binlogdata.VStreamRowsResponse \ + --go-vtproto_opt=pool=vitess.io/vitess/go/vt/proto/binlogdata.VStreamTablesResponse \ -I${PWD}/dist/vt-protoc-21.3/include:proto $(PROTO_SRCS) cp -Rf vitess.io/vitess/go/vt/proto/* go/vt/proto rm -rf vitess.io/vitess/go/vt/proto/ diff --git a/go/cmd/vtctldclient/command/movetables.go b/go/cmd/vtctldclient/command/movetables.go index 7eaff741a45..964e684ee44 100644 --- a/go/cmd/vtctldclient/command/movetables.go +++ b/go/cmd/vtctldclient/command/movetables.go @@ -79,6 +79,24 @@ See the --help output for each command for more details.`, RunE: commandMoveTablesComplete, } + checkAtomicCopyOptions = func() error { + var errors []string + if !moveTablesCreateOptions.AtomicCopy { + return nil + } + if !moveTablesCreateOptions.AllTables { + errors = append(errors, "atomic copy requires --all-tables.") + } + if len(moveTablesCreateOptions.IncludeTables) > 0 || len(moveTablesCreateOptions.ExcludeTables) > 0 { + errors = append(errors, "atomic copy does not support specifying tables.") + } + if len(errors) > 0 { + errors = append(errors, "Found options incompatible with atomic copy:") + return fmt.Errorf(strings.Join(errors, " ")) + } + return nil + } + // MoveTablesCreate makes a MoveTablesCreate gRPC call to a vtctld. MoveTablesCreate = &cobra.Command{ Use: "create", @@ -89,6 +107,7 @@ See the --help output for each command for more details.`, Aliases: []string{"Create"}, Args: cobra.NoArgs, PreRunE: func(cmd *cobra.Command, args []string) error { + // Either specific tables or the all tables flags are required. if !cmd.Flags().Lookup("tables").Changed && !cmd.Flags().Lookup("all-tables").Changed { return fmt.Errorf("tables or all-tables are required to specify which tables to move") @@ -104,6 +123,10 @@ See the --help output for each command for more details.`, if _, ok := binlogdatapb.OnDDLAction_value[strings.ToUpper(moveTablesCreateOptions.OnDDL)]; !ok { return fmt.Errorf("invalid on-ddl value: %s", moveTablesCreateOptions.OnDDL) } + + if err := checkAtomicCopyOptions(); err != nil { + return err + } return nil }, RunE: commandMoveTablesCreate, @@ -235,6 +258,7 @@ var ( AutoStart bool StopAfterCopy bool NoRoutingRules bool + AtomicCopy bool }{} moveTablesSwitchTrafficOptions = struct { Cells []string @@ -285,6 +309,7 @@ func commandMoveTablesCreate(cmd *cobra.Command, args []string) error { AutoStart: moveTablesCreateOptions.AutoStart, StopAfterCopy: moveTablesCreateOptions.StopAfterCopy, NoRoutingRules: moveTablesCreateOptions.NoRoutingRules, + AtomicCopy: moveTablesCreateOptions.AtomicCopy, } resp, err := client.MoveTablesCreate(commandCtx, req) @@ -527,6 +552,7 @@ func init() { MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.AutoStart, "auto-start", true, "Start the MoveTables workflow after creating it") MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.StopAfterCopy, "stop-after-copy", false, "Stop the MoveTables workflow after it's finished copying the existing rows and before it starts replicating changes") MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.NoRoutingRules, "no-routing-rules", false, "(Advanced) Do not create routing rules while creating the workflow. See the reference documentation for limitations if you use this flag.") + MoveTablesCreate.Flags().BoolVar(&moveTablesCreateOptions.AtomicCopy, "atomic-copy", false, "(EXPERIMENTAL) A single copy phase is run for all tables from the source. Use this, for example, if your source keyspace has tables which use foreign key constraints.") MoveTables.AddCommand(MoveTablesCreate) MoveTables.AddCommand(MoveTablesShow) diff --git a/go/cmd/vtctldclient/command/workflows.go b/go/cmd/vtctldclient/command/workflows.go index a67bd1e43dc..a9ef6df88c7 100644 --- a/go/cmd/vtctldclient/command/workflows.go +++ b/go/cmd/vtctldclient/command/workflows.go @@ -223,6 +223,42 @@ func commandWorkflowDelete(cmd *cobra.Command, args []string) error { return nil } +func getWorkflow(keyspace, workflow string) (*vtctldatapb.GetWorkflowsResponse, error) { + resp, err := client.GetWorkflows(commandCtx, &vtctldatapb.GetWorkflowsRequest{ + Keyspace: keyspace, + Workflow: workflow, + }) + if err != nil { + return &vtctldatapb.GetWorkflowsResponse{}, err + } + return resp, nil +} + +// canStartWorkflow validates that, for an atomic copy workflow, none of the streams are still in the copy phase. +// Since we copy all tables in a single snapshot, we cannot restart a workflow which broke before all tables were copied. +func canStartWorkflow(keyspace, workflow string) error { + resp, err := getWorkflow(keyspace, workflow) + if err != nil { + return err + } + if len(resp.Workflows) == 0 { + return fmt.Errorf("workflow %s not found", workflow) + } + wf := resp.Workflows[0] + if wf.WorkflowSubType != binlogdatapb.VReplicationWorkflowSubType_AtomicCopy.String() { + return nil + } + // If we're here, we have an atomic copy workflow. + for _, shardStream := range wf.ShardStreams { + for _, stream := range shardStream.Streams { + if len(stream.CopyStates) > 0 { + return fmt.Errorf("stream %d is still in the copy phase: can only start workflow %s if all streams have completed the copy phase.", stream.Id, workflow) + } + } + } + return nil +} + func commandWorkflowShow(cmd *cobra.Command, args []string) error { cli.FinishedParsing(cmd) @@ -312,6 +348,10 @@ func commandWorkflowUpdateState(cmd *cobra.Command, args []string) error { var state binlogdatapb.VReplicationWorkflowState switch strings.ToLower(cmd.Name()) { case "start": + if err := canStartWorkflow(workflowUpdateOptions.Workflow, workflowOptions.Keyspace); err != nil { + return err + } + state = binlogdatapb.VReplicationWorkflowState_Running case "stop": state = binlogdatapb.VReplicationWorkflowState_Stopped diff --git a/go/test/endtoend/vreplication/cluster_test.go b/go/test/endtoend/vreplication/cluster_test.go index 9861f71f6fd..1fd0aabc822 100644 --- a/go/test/endtoend/vreplication/cluster_test.go +++ b/go/test/endtoend/vreplication/cluster_test.go @@ -89,6 +89,7 @@ type ClusterConfig struct { // VitessCluster represents all components within the test cluster type VitessCluster struct { + t *testing.T ClusterConfig *ClusterConfig Name string Cells map[string]*Cell @@ -320,7 +321,7 @@ func init() { // NewVitessCluster starts a basic cluster with vtgate, vtctld and the topo func NewVitessCluster(t *testing.T, name string, cellNames []string, clusterConfig *ClusterConfig) *VitessCluster { - vc := &VitessCluster{Name: name, Cells: make(map[string]*Cell), ClusterConfig: clusterConfig} + vc := &VitessCluster{t: t, Name: name, Cells: make(map[string]*Cell), ClusterConfig: clusterConfig} require.NotNil(t, vc) vc.CleanupDataroot(t, true) diff --git a/go/test/endtoend/vreplication/fk_config_test.go b/go/test/endtoend/vreplication/fk_config_test.go new file mode 100644 index 00000000000..5b02aeb62bb --- /dev/null +++ b/go/test/endtoend/vreplication/fk_config_test.go @@ -0,0 +1,65 @@ +/* +Copyright 2023 The Vitess Authors. + +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 vreplication + +var ( + initialFKSchema = ` +create table parent(id int, name varchar(128), primary key(id)) engine=innodb; +create table child(id int, parent_id int, name varchar(128), primary key(id), foreign key(parent_id) references parent(id) on delete cascade) engine=innodb; +` + initialFKData = ` +insert into parent values(1, 'parent1'), (2, 'parent2'); +insert into child values(1, 1, 'child11'), (2, 1, 'child21'), (3, 2, 'child32');` + + initialFKSourceVSchema = ` +{ + "tables": { + "parent": {}, + "child": {} + } +} +` + + initialFKTargetVSchema = ` +{ + "sharded": true, + "vindexes": { + "reverse_bits": { + "type": "reverse_bits" + } + }, + "tables": { + "parent": { + "column_vindexes": [ + { + "column": "id", + "name": "reverse_bits" + } + ] + }, + "child": { + "column_vindexes": [ + { + "column": "parent_id", + "name": "reverse_bits" + } + ] + } + } +} +` +) diff --git a/go/test/endtoend/vreplication/fk_test.go b/go/test/endtoend/vreplication/fk_test.go new file mode 100644 index 00000000000..f96e0ddd8ac --- /dev/null +++ b/go/test/endtoend/vreplication/fk_test.go @@ -0,0 +1,256 @@ +/* +Copyright 2023 The Vitess Authors. + +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 vreplication + +import ( + "context" + "fmt" + "math/rand" + "strconv" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/vt/log" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +// TestFKWorkflow runs a MoveTables workflow with atomic copy for a db with foreign key constraints. +// It inserts initial data, then simulates load. We insert both child rows with foreign keys and those without, +// i.e. with foreign_key_checks=0. +func TestFKWorkflow(t *testing.T) { + cellName := "zone" + cells := []string{cellName} + vc = NewVitessCluster(t, "TestFKWorkflow", cells, mainClusterConfig) + require.NotNil(t, vc) + allCellNames = cellName + defaultCellName := cellName + defaultCell = vc.Cells[defaultCellName] + sourceKeyspace := "fksource" + shardName := "0" + + defer vc.TearDown(t) + + cell := vc.Cells[cellName] + vc.AddKeyspace(t, []*Cell{cell}, sourceKeyspace, shardName, initialFKSourceVSchema, initialFKSchema, 0, 0, 100, sourceKsOpts) + + vtgate = cell.Vtgates[0] + require.NotNil(t, vtgate) + err := cluster.WaitForHealthyShard(vc.VtctldClient, sourceKeyspace, shardName) + require.NoError(t, err) + vtgate.WaitForStatusOfTabletInShard(fmt.Sprintf("%s.%s.primary", sourceKeyspace, shardName), 1, 30*time.Second) + + vtgateConn = getConnection(t, vc.ClusterConfig.hostname, vc.ClusterConfig.vtgateMySQLPort) + defer vtgateConn.Close() + verifyClusterHealth(t, vc) + + insertInitialFKData(t) + withLoad := true // Set it to false to skip load simulation, while debugging + var cancel context.CancelFunc + var ctx context.Context + if withLoad { + ctx, cancel = context.WithCancel(context.Background()) + defer func() { + select { + case <-ctx.Done(): + default: + cancel() + } + }() + go simulateLoad(t, ctx) + } + targetKeyspace := "fktarget" + targetTabletId := 200 + vc.AddKeyspace(t, []*Cell{cell}, targetKeyspace, shardName, initialFKTargetVSchema, initialFKSchema, 0, 0, targetTabletId, sourceKsOpts) + vtgate.WaitForStatusOfTabletInShard(fmt.Sprintf("%s.%s.primary", targetKeyspace, shardName), 1, 30*time.Second) + + workflowName := "fk" + ksWorkflow := fmt.Sprintf("%s.%s", targetKeyspace, workflowName) + + mt := newMoveTables(vc, &moveTables{ + workflowName: workflowName, + targetKeyspace: targetKeyspace, + sourceKeyspace: sourceKeyspace, + atomicCopy: true, + }, moveTablesFlavorRandom) + mt.Create() + + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) + targetKs := vc.Cells[cellName].Keyspaces[targetKeyspace] + targetTab := targetKs.Shards["0"].Tablets[fmt.Sprintf("%s-%d", cellName, targetTabletId)].Vttablet + require.NotNil(t, targetTab) + catchup(t, targetTab, workflowName, "MoveTables") + vdiff(t, targetKeyspace, workflowName, cellName, true, false, nil) + waitForAdditionalRows(t, 200) + vdiff(t, targetKeyspace, workflowName, cellName, true, false, nil) + if withLoad { + go func() { + cancel() + }() + <-ch + } + mt.SwitchReadsAndWrites() + + log.Infof("Switch traffic done") + + if withLoad { + ctx, cancel = context.WithCancel(context.Background()) + defer func() { + select { + case <-ctx.Done(): + default: + cancel() + } + }() + go simulateLoad(t, ctx) + } + waitForAdditionalRows(t, 200) + if withLoad { + go func() { + cancel() + }() + <-ch + } +} + +func insertInitialFKData(t *testing.T) { + t.Run("insertInitialFKData", func(t *testing.T) { + sourceKeyspace := "fksource" + shard := "0" + db := fmt.Sprintf("%s:%s", sourceKeyspace, shard) + log.Infof("Inserting initial FK data") + execMultipleQueries(t, vtgateConn, db, initialFKData) + log.Infof("Done inserting initial FK data") + waitForRowCount(t, vtgateConn, db, "parent", 2) + waitForRowCount(t, vtgateConn, db, "child", 3) + }) +} + +var currentParentId int64 +var currentChildId int64 + +func init() { + currentParentId = 100 + currentChildId = 100 +} + +var ch = make(chan bool) + +func simulateLoad(t *testing.T, ctx context.Context) { + var err error + for i := 0; ; i++ { + if i%1000 == 0 { + log.Infof("Load simulation iteration %d", i) + } + select { + case <-ctx.Done(): + ch <- true + return + default: + } + // Decide operation based on random number + op := rand.Intn(100) + switch { + case op < 50: // 50% chance to insert + insert(t) + case op < 80: // 30% chance to update + update(t) + default: // 20% chance to delete + delete(t) + } + require.NoError(t, err) + time.Sleep(1 * time.Millisecond) + } +} + +func getNumRowsParent(t *testing.T, vtgateConn *mysql.Conn) int { + qr := execVtgateQuery(t, vtgateConn, "fksource", "SELECT COUNT(*) FROM parent") + require.NotNil(t, qr) + numRows, err := strconv.Atoi(qr.Rows[0][0].ToString()) + require.NoError(t, err) + return numRows +} + +func waitForAdditionalRows(t *testing.T, count int) { + vtgateConn := getConnection(t, vc.ClusterConfig.hostname, vc.ClusterConfig.vtgateMySQLPort) + defer vtgateConn.Close() + numRowsStart := getNumRowsParent(t, vtgateConn) + shortCtx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + for { + switch { + case shortCtx.Err() != nil: + t.Fatalf("Timed out waiting for additional rows") + default: + numRows := getNumRowsParent(t, vtgateConn) + if numRows >= numRowsStart+count { + return + } + time.Sleep(10 * time.Millisecond) + } + } +} + +func exec2(t *testing.T, query string) *sqltypes.Result { + qr := execVtgateQuery(t, vtgateConn, "fksource", query) + require.NotNil(t, qr) + return qr +} + +func insert(t *testing.T) { + currentParentId++ + insertQuery := fmt.Sprintf("INSERT INTO parent (id) VALUES (%d)", currentParentId) + qr := exec2(t, insertQuery) + require.NotNil(t, qr) + for i := 0; i < rand.Intn(4)+1; i++ { + currentChildId++ + if i == 3 { + insertQuery = fmt.Sprintf("INSERT /*+ SET_VAR(foreign_key_checks=0) */ INTO child (id, parent_id) VALUES (%d, %d)", currentChildId, currentParentId+1000000) + exec2(t, insertQuery) + } else { + insertQuery = fmt.Sprintf("INSERT INTO child (id, parent_id) VALUES (%d, %d)", currentChildId, currentParentId) + exec2(t, insertQuery) + } + } +} + +func getRandomId(t *testing.T) int64 { + selectQuery := "SELECT id FROM parent ORDER BY RAND() LIMIT 1" + qr := exec2(t, selectQuery) + require.NotNil(t, qr) + if len(qr.Rows) == 0 { + return 0 + } + id, err := qr.Rows[0][0].ToInt64() + require.NoError(t, err) + return id +} + +func update(t *testing.T) { + updateQuery := fmt.Sprintf("UPDATE parent SET name = 'parent%d' WHERE id = %d", rand.Intn(1000)+1, getRandomId(t)) + exec2(t, updateQuery) +} + +func delete(t *testing.T) { + deleteQuery := fmt.Sprintf("DELETE FROM parent WHERE id = %d", getRandomId(t)) + exec2(t, deleteQuery) +} diff --git a/go/test/endtoend/vreplication/movetables_buffering_test.go b/go/test/endtoend/vreplication/movetables_buffering_test.go index 2dc2db7b909..f2e60cb0db1 100644 --- a/go/test/endtoend/vreplication/movetables_buffering_test.go +++ b/go/test/endtoend/vreplication/movetables_buffering_test.go @@ -21,7 +21,7 @@ func TestMoveTablesBuffering(t *testing.T) { setupMinimalCustomerKeyspace(t) tables := "loadtest" err := tstWorkflowExec(t, defaultCellName, workflowName, sourceKs, targetKs, - tables, workflowActionCreate, "", "", "") + tables, workflowActionCreate, "", "", "", false) require.NoError(t, err) waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) diff --git a/go/test/endtoend/vreplication/partial_movetables_seq_test.go b/go/test/endtoend/vreplication/partial_movetables_seq_test.go index e08561909af..36e73f80f31 100644 --- a/go/test/endtoend/vreplication/partial_movetables_seq_test.go +++ b/go/test/endtoend/vreplication/partial_movetables_seq_test.go @@ -239,7 +239,7 @@ func (wf *workflow) create() { currentWorkflowType = wrangler.MoveTablesWorkflow sourceShards := strings.Join(wf.options.sourceShards, ",") err = tstWorkflowExec(t, cell, wf.name, wf.fromKeyspace, wf.toKeyspace, - strings.Join(wf.options.tables, ","), workflowActionCreate, "", sourceShards, "") + strings.Join(wf.options.tables, ","), workflowActionCreate, "", sourceShards, "", false) case "reshard": currentWorkflowType = wrangler.ReshardWorkflow sourceShards := strings.Join(wf.options.sourceShards, ",") @@ -248,7 +248,7 @@ func (wf *workflow) create() { targetShards = sourceShards } err = tstWorkflowExec(t, cell, wf.name, wf.fromKeyspace, wf.toKeyspace, - strings.Join(wf.options.tables, ","), workflowActionCreate, "", sourceShards, targetShards) + strings.Join(wf.options.tables, ","), workflowActionCreate, "", sourceShards, targetShards, false) default: panic(fmt.Sprintf("unknown workflow type: %s", wf.typ)) } @@ -266,15 +266,15 @@ func (wf *workflow) create() { } func (wf *workflow) switchTraffic() { - require.NoError(wf.tc.t, tstWorkflowExec(wf.tc.t, wf.tc.defaultCellName, wf.name, wf.fromKeyspace, wf.toKeyspace, "", workflowActionSwitchTraffic, "", "", "")) + require.NoError(wf.tc.t, tstWorkflowExec(wf.tc.t, wf.tc.defaultCellName, wf.name, wf.fromKeyspace, wf.toKeyspace, "", workflowActionSwitchTraffic, "", "", "", false)) } func (wf *workflow) reverseTraffic() { - require.NoError(wf.tc.t, tstWorkflowExec(wf.tc.t, wf.tc.defaultCellName, wf.name, wf.fromKeyspace, wf.toKeyspace, "", workflowActionReverseTraffic, "", "", "")) + require.NoError(wf.tc.t, tstWorkflowExec(wf.tc.t, wf.tc.defaultCellName, wf.name, wf.fromKeyspace, wf.toKeyspace, "", workflowActionReverseTraffic, "", "", "", false)) } func (wf *workflow) complete() { - require.NoError(wf.tc.t, tstWorkflowExec(wf.tc.t, wf.tc.defaultCellName, wf.name, wf.fromKeyspace, wf.toKeyspace, "", workflowActionComplete, "", "", "")) + require.NoError(wf.tc.t, tstWorkflowExec(wf.tc.t, wf.tc.defaultCellName, wf.name, wf.fromKeyspace, wf.toKeyspace, "", workflowActionComplete, "", "", "", false)) } // TestPartialMoveTablesWithSequences enhances TestPartialMoveTables by adding an unsharded keyspace which has a @@ -505,7 +505,7 @@ func TestPartialMoveTablesWithSequences(t *testing.T) { // We switched traffic, so it's the reverse workflow we want to cancel. reverseWf := wf + "_reverse" reverseKs := sourceKs // customer - err = tstWorkflowExec(t, "", reverseWf, "", reverseKs, "", workflowActionCancel, "", "", "") + err = tstWorkflowExec(t, "", reverseWf, "", reverseKs, "", workflowActionCancel, "", "", "", false) require.NoError(t, err) output, err := tc.vc.VtctlClient.ExecuteCommandWithOutput("Workflow", fmt.Sprintf("%s.%s", reverseKs, reverseWf), "show") diff --git a/go/test/endtoend/vreplication/partial_movetables_test.go b/go/test/endtoend/vreplication/partial_movetables_test.go index 321d6afc6c1..7c0784255d7 100644 --- a/go/test/endtoend/vreplication/partial_movetables_test.go +++ b/go/test/endtoend/vreplication/partial_movetables_test.go @@ -90,7 +90,7 @@ func TestPartialMoveTablesBasic(t *testing.T) { // start the partial movetables for 80- err := tstWorkflowExec(t, defaultCellName, wfName, sourceKs, targetKs, - "customer,loadtest", workflowActionCreate, "", shard, "") + "customer,loadtest", workflowActionCreate, "", shard, "", false) require.NoError(t, err) var lg *loadGenerator if runWithLoad { // start load after routing rules are set, otherwise we end up with ambiguous tables @@ -163,7 +163,7 @@ func TestPartialMoveTablesBasic(t *testing.T) { require.Contains(t, err.Error(), "target: customer.-80.primary", "Query was routed to the target before any SwitchTraffic") // Switch all traffic for the shard - require.NoError(t, tstWorkflowExec(t, "", wfName, "", targetKs, "", workflowActionSwitchTraffic, "", "", "")) + require.NoError(t, tstWorkflowExec(t, "", wfName, "", targetKs, "", workflowActionSwitchTraffic, "", "", "", false)) expectedSwitchOutput := fmt.Sprintf("SwitchTraffic was successful for workflow %s.%s\nStart State: Reads Not Switched. Writes Not Switched\nCurrent State: Reads partially switched, for shards: %s. Writes partially switched, for shards: %s\n\n", targetKs, wfName, shard, shard) require.Equal(t, expectedSwitchOutput, lastOutput) @@ -221,7 +221,7 @@ func TestPartialMoveTablesBasic(t *testing.T) { // We cannot Complete a partial move tables at the moment because // it will find that all traffic has (obviously) not been switched. - err = tstWorkflowExec(t, "", wfName, "", targetKs, "", workflowActionComplete, "", "", "") + err = tstWorkflowExec(t, "", wfName, "", targetKs, "", workflowActionComplete, "", "", "", false) require.Error(t, err) // Confirm global routing rules: -80 should still be be routed to customer @@ -234,14 +234,14 @@ func TestPartialMoveTablesBasic(t *testing.T) { ksWf = fmt.Sprintf("%s.%s", targetKs, wfName) // Start the partial movetables for -80, 80- has already been switched err = tstWorkflowExec(t, defaultCellName, wfName, sourceKs, targetKs, - "customer,loadtest", workflowActionCreate, "", shard, "") + "customer,loadtest", workflowActionCreate, "", shard, "", false) require.NoError(t, err) targetTab2 := vc.getPrimaryTablet(t, targetKs, shard) catchup(t, targetTab2, wfName, "Partial MoveTables Customer to Customer2: -80") vdiff1(t, ksWf, "") // Switch all traffic for the shard - require.NoError(t, tstWorkflowExec(t, "", wfName, "", targetKs, "", workflowActionSwitchTraffic, "", "", "")) + require.NoError(t, tstWorkflowExec(t, "", wfName, "", targetKs, "", workflowActionSwitchTraffic, "", "", "", false)) expectedSwitchOutput = fmt.Sprintf("SwitchTraffic was successful for workflow %s.%s\nStart State: Reads partially switched, for shards: 80-. Writes partially switched, for shards: 80-\nCurrent State: All Reads Switched. All Writes Switched\n\n", targetKs, wfName) require.Equal(t, expectedSwitchOutput, lastOutput) @@ -262,7 +262,7 @@ func TestPartialMoveTablesBasic(t *testing.T) { // We switched traffic, so it's the reverse workflow we want to cancel. reverseWf := wf + "_reverse" reverseKs := sourceKs // customer - err = tstWorkflowExec(t, "", reverseWf, "", reverseKs, "", workflowActionCancel, "", "", "") + err = tstWorkflowExec(t, "", reverseWf, "", reverseKs, "", workflowActionCancel, "", "", "", false) require.NoError(t, err) output, err := vc.VtctlClient.ExecuteCommandWithOutput("Workflow", fmt.Sprintf("%s.%s", reverseKs, reverseWf), "show") diff --git a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go index c3a79c58ac1..ec20e1d92ca 100644 --- a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go +++ b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go @@ -63,7 +63,7 @@ var ( func createReshardWorkflow(t *testing.T, sourceShards, targetShards string) error { err := tstWorkflowExec(t, defaultCellName, workflowName, targetKs, targetKs, - "", workflowActionCreate, "", sourceShards, targetShards) + "", workflowActionCreate, "", sourceShards, targetShards, false) require.NoError(t, err) waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) confirmTablesHaveSecondaryKeys(t, []*cluster.VttabletProcess{targetTab1}, targetKs, "") @@ -78,7 +78,7 @@ func createMoveTablesWorkflow(t *testing.T, tables string) { tables = tablesToMove } err := tstWorkflowExec(t, defaultCellName, workflowName, sourceKs, targetKs, - tables, workflowActionCreate, "", "", "") + tables, workflowActionCreate, "", "", "", false) require.NoError(t, err) waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) confirmTablesHaveSecondaryKeys(t, []*cluster.VttabletProcess{targetTab1}, targetKs, tables) @@ -88,10 +88,10 @@ func createMoveTablesWorkflow(t *testing.T, tables string) { } func tstWorkflowAction(t *testing.T, action, tabletTypes, cells string) error { - return tstWorkflowExec(t, cells, workflowName, sourceKs, targetKs, tablesToMove, action, tabletTypes, "", "") + return tstWorkflowExec(t, cells, workflowName, sourceKs, targetKs, tablesToMove, action, tabletTypes, "", "", false) } -func tstWorkflowExec(t *testing.T, cells, workflow, sourceKs, targetKs, tables, action, tabletTypes, sourceShards, targetShards string) error { +func tstWorkflowExec(t *testing.T, cells, workflow, sourceKs, targetKs, tables, action, tabletTypes, sourceShards, targetShards string, atomicCopy bool) error { var args []string if currentWorkflowType == wrangler.MoveTablesWorkflow { args = append(args, "MoveTables") @@ -104,11 +104,18 @@ func tstWorkflowExec(t *testing.T, cells, workflow, sourceKs, targetKs, tables, if BypassLagCheck { args = append(args, "--max_replication_lag_allowed=2542087h") } - + if atomicCopy { + args = append(args, "--atomic-copy") + } switch action { case workflowActionCreate: if currentWorkflowType == wrangler.MoveTablesWorkflow { - args = append(args, "--source", sourceKs, "--tables", tables) + args = append(args, "--source", sourceKs) + if tables != "" { + args = append(args, "--tables", tables) + } else { + args = append(args, "--all") + } if sourceShards != "" { args = append(args, "--source_shards", sourceShards) } @@ -118,7 +125,9 @@ func tstWorkflowExec(t *testing.T, cells, workflow, sourceKs, targetKs, tables, // Test new experimental --defer-secondary-keys flag switch currentWorkflowType { case wrangler.MoveTablesWorkflow, wrangler.MigrateWorkflow, wrangler.ReshardWorkflow: - args = append(args, "--defer-secondary-keys") + if !atomicCopy { + args = append(args, "--defer-secondary-keys") + } args = append(args, "--initialize-target-sequences") // Only used for MoveTables } } @@ -308,17 +317,17 @@ func testVSchemaForSequenceAfterMoveTables(t *testing.T) { // use MoveTables to move customer2 from product to customer using currentWorkflowType = wrangler.MoveTablesWorkflow err := tstWorkflowExec(t, defaultCellName, "wf2", sourceKs, targetKs, - "customer2", workflowActionCreate, "", "", "") + "customer2", workflowActionCreate, "", "", "", false) require.NoError(t, err) waitForWorkflowState(t, vc, "customer.wf2", binlogdatapb.VReplicationWorkflowState_Running.String()) waitForLowLag(t, "customer", "wf2") err = tstWorkflowExec(t, defaultCellName, "wf2", sourceKs, targetKs, - "", workflowActionSwitchTraffic, "", "", "") + "", workflowActionSwitchTraffic, "", "", "", false) require.NoError(t, err) err = tstWorkflowExec(t, defaultCellName, "wf2", sourceKs, targetKs, - "", workflowActionComplete, "", "", "") + "", workflowActionComplete, "", "", "", false) require.NoError(t, err) // sanity check @@ -343,16 +352,16 @@ func testVSchemaForSequenceAfterMoveTables(t *testing.T) { // use MoveTables to move customer2 back to product. Note that now the table has an associated sequence err = tstWorkflowExec(t, defaultCellName, "wf3", targetKs, sourceKs, - "customer2", workflowActionCreate, "", "", "") + "customer2", workflowActionCreate, "", "", "", false) require.NoError(t, err) waitForWorkflowState(t, vc, "product.wf3", binlogdatapb.VReplicationWorkflowState_Running.String()) waitForLowLag(t, "product", "wf3") err = tstWorkflowExec(t, defaultCellName, "wf3", targetKs, sourceKs, - "", workflowActionSwitchTraffic, "", "", "") + "", workflowActionSwitchTraffic, "", "", "", false) require.NoError(t, err) err = tstWorkflowExec(t, defaultCellName, "wf3", targetKs, sourceKs, - "", workflowActionComplete, "", "", "") + "", workflowActionComplete, "", "", "", false) require.NoError(t, err) // sanity check diff --git a/go/test/endtoend/vreplication/wrappers_test.go b/go/test/endtoend/vreplication/wrappers_test.go new file mode 100644 index 00000000000..3d06c5029a6 --- /dev/null +++ b/go/test/endtoend/vreplication/wrappers_test.go @@ -0,0 +1,191 @@ +/* +Copyright 2023 The Vitess Authors. + +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 vreplication + +import ( + "math/rand" + "strconv" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/vt/log" +) + +type moveTablesFlavor int + +const ( + moveTablesFlavorRandom moveTablesFlavor = iota + moveTablesFlavorVtctl + moveTablesFlavorVtctld +) + +var moveTablesFlavors = []moveTablesFlavor{ + moveTablesFlavorVtctl, + moveTablesFlavorVtctld, +} + +type moveTables struct { + vc *VitessCluster + workflowName string + targetKeyspace string + sourceKeyspace string + tables string + atomicCopy bool +} + +type iMoveTables interface { + Create() + Show() + SwitchReads() + SwitchWrites() + SwitchReadsAndWrites() + Cancel() + Complete() + Flavor() string +} + +func newMoveTables(vc *VitessCluster, mt *moveTables, flavor moveTablesFlavor) iMoveTables { + mt.vc = vc + var mt2 iMoveTables + if flavor == moveTablesFlavorRandom { + flavor = moveTablesFlavors[rand.Intn(len(moveTablesFlavors))] + } + switch flavor { + case moveTablesFlavorVtctl: + mt2 = newVtctlMoveTables(mt) + case moveTablesFlavorVtctld: + mt2 = newVtctldMoveTables(mt) + default: + panic("unreachable") + } + log.Infof("Using moveTables flavor: %s", mt2.Flavor()) + return mt2 +} + +type VtctlMoveTables struct { + *moveTables +} + +func (vmt *VtctlMoveTables) Flavor() string { + return "vtctl" +} + +func newVtctlMoveTables(mt *moveTables) *VtctlMoveTables { + return &VtctlMoveTables{mt} +} + +func (vmt *VtctlMoveTables) Create() { + log.Infof("vmt is %+v", vmt.vc, vmt.tables) + err := tstWorkflowExec(vmt.vc.t, "", vmt.workflowName, vmt.sourceKeyspace, vmt.targetKeyspace, + vmt.tables, workflowActionCreate, "", "", "", vmt.atomicCopy) + require.NoError(vmt.vc.t, err) +} + +func (vmt *VtctlMoveTables) SwitchReadsAndWrites() { + err := tstWorkflowExec(vmt.vc.t, "", vmt.workflowName, vmt.sourceKeyspace, vmt.targetKeyspace, + vmt.tables, workflowActionSwitchTraffic, "", "", "", vmt.atomicCopy) + require.NoError(vmt.vc.t, err) +} + +func (vmt *VtctlMoveTables) Show() { + //TODO implement me + panic("implement me") +} + +func (vmt *VtctlMoveTables) SwitchReads() { + //TODO implement me + panic("implement me") +} + +func (vmt *VtctlMoveTables) SwitchWrites() { + //TODO implement me + panic("implement me") +} + +func (vmt *VtctlMoveTables) Cancel() { + //TODO implement me + panic("implement me") +} + +func (vmt *VtctlMoveTables) Complete() { + //TODO implement me + panic("implement me") +} + +var _ iMoveTables = (*VtctldMoveTables)(nil) + +type VtctldMoveTables struct { + *moveTables +} + +func newVtctldMoveTables(mt *moveTables) *VtctldMoveTables { + return &VtctldMoveTables{mt} +} + +func (v VtctldMoveTables) Flavor() string { + return "vtctld" +} + +func (v VtctldMoveTables) exec(args ...string) { + args2 := []string{"MoveTables", "--workflow=" + v.workflowName, "--target-keyspace=" + v.targetKeyspace} + args2 = append(args2, args...) + if err := vc.VtctldClient.ExecuteCommand(args2...); err != nil { + v.vc.t.Fatalf("failed to create MoveTables workflow: %v", err) + } +} + +func (v VtctldMoveTables) Create() { + args := []string{"Create", "--source-keyspace=" + v.sourceKeyspace} + if v.tables != "" { + args = append(args, "--tables="+v.tables) + } else { + args = append(args, "--all-tables") + } + if v.atomicCopy { + args = append(args, "--atomic-copy="+strconv.FormatBool(v.atomicCopy)) + } + v.exec(args...) +} + +func (v VtctldMoveTables) SwitchReadsAndWrites() { + v.exec("SwitchTraffic") +} + +func (v VtctldMoveTables) Show() { + //TODO implement me + panic("implement me") +} + +func (v VtctldMoveTables) SwitchReads() { + //TODO implement me + panic("implement me") +} + +func (v VtctldMoveTables) SwitchWrites() { + //TODO implement me + panic("implement me") +} + +func (v VtctldMoveTables) Cancel() { + //TODO implement me + panic("implement me") +} + +func (v VtctldMoveTables) Complete() { + //TODO implement me + panic("implement me") +} diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index 66a9b2c6d92..3da747d3832 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -158,8 +158,9 @@ func (VReplicationWorkflowType) EnumDescriptor() ([]byte, []int) { type VReplicationWorkflowSubType int32 const ( - VReplicationWorkflowSubType_None VReplicationWorkflowSubType = 0 - VReplicationWorkflowSubType_Partial VReplicationWorkflowSubType = 1 + VReplicationWorkflowSubType_None VReplicationWorkflowSubType = 0 + VReplicationWorkflowSubType_Partial VReplicationWorkflowSubType = 1 + VReplicationWorkflowSubType_AtomicCopy VReplicationWorkflowSubType = 2 ) // Enum value maps for VReplicationWorkflowSubType. @@ -167,10 +168,12 @@ var ( VReplicationWorkflowSubType_name = map[int32]string{ 0: "None", 1: "Partial", + 2: "AtomicCopy", } VReplicationWorkflowSubType_value = map[string]int32{ - "None": 0, - "Partial": 1, + "None": 0, + "Partial": 1, + "AtomicCopy": 2, } ) @@ -1408,6 +1411,7 @@ type RowEvent struct { RowChanges []*RowChange `protobuf:"bytes,2,rep,name=row_changes,json=rowChanges,proto3" json:"row_changes,omitempty"` Keyspace string `protobuf:"bytes,3,opt,name=keyspace,proto3" json:"keyspace,omitempty"` Shard string `protobuf:"bytes,4,opt,name=shard,proto3" json:"shard,omitempty"` + Flags uint32 `protobuf:"varint,5,opt,name=flags,proto3" json:"flags,omitempty"` // https://dev.mysql.com/doc/dev/mysql-server/latest/classbinary__log_1_1Rows__event.html } func (x *RowEvent) Reset() { @@ -1470,6 +1474,13 @@ func (x *RowEvent) GetShard() string { return "" } +func (x *RowEvent) GetFlags() uint32 { + if x != nil { + return x.Flags + } + return 0 +} + // FieldEvent represents the field info for a table. type FieldEvent struct { state protoimpl.MessageState @@ -2434,6 +2445,158 @@ func (x *VStreamRowsResponse) GetHeartbeat() bool { return false } +// VStreamTablesRequest is the payload for VStreamTables +type VStreamTablesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + EffectiveCallerId *vtrpc.CallerID `protobuf:"bytes,1,opt,name=effective_caller_id,json=effectiveCallerId,proto3" json:"effective_caller_id,omitempty"` + ImmediateCallerId *query.VTGateCallerID `protobuf:"bytes,2,opt,name=immediate_caller_id,json=immediateCallerId,proto3" json:"immediate_caller_id,omitempty"` + Target *query.Target `protobuf:"bytes,3,opt,name=target,proto3" json:"target,omitempty"` +} + +func (x *VStreamTablesRequest) Reset() { + *x = VStreamTablesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_binlogdata_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VStreamTablesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VStreamTablesRequest) ProtoMessage() {} + +func (x *VStreamTablesRequest) ProtoReflect() protoreflect.Message { + mi := &file_binlogdata_proto_msgTypes[24] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VStreamTablesRequest.ProtoReflect.Descriptor instead. +func (*VStreamTablesRequest) Descriptor() ([]byte, []int) { + return file_binlogdata_proto_rawDescGZIP(), []int{24} +} + +func (x *VStreamTablesRequest) GetEffectiveCallerId() *vtrpc.CallerID { + if x != nil { + return x.EffectiveCallerId + } + return nil +} + +func (x *VStreamTablesRequest) GetImmediateCallerId() *query.VTGateCallerID { + if x != nil { + return x.ImmediateCallerId + } + return nil +} + +func (x *VStreamTablesRequest) GetTarget() *query.Target { + if x != nil { + return x.Target + } + return nil +} + +// VStreamTablesResponse is the response from VStreamTables +type VStreamTablesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TableName string `protobuf:"bytes,1,opt,name=table_name,json=tableName,proto3" json:"table_name,omitempty"` + Fields []*query.Field `protobuf:"bytes,2,rep,name=fields,proto3" json:"fields,omitempty"` + Pkfields []*query.Field `protobuf:"bytes,3,rep,name=pkfields,proto3" json:"pkfields,omitempty"` + Gtid string `protobuf:"bytes,4,opt,name=gtid,proto3" json:"gtid,omitempty"` + Rows []*query.Row `protobuf:"bytes,5,rep,name=rows,proto3" json:"rows,omitempty"` + Lastpk *query.Row `protobuf:"bytes,6,opt,name=lastpk,proto3" json:"lastpk,omitempty"` +} + +func (x *VStreamTablesResponse) Reset() { + *x = VStreamTablesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_binlogdata_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *VStreamTablesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VStreamTablesResponse) ProtoMessage() {} + +func (x *VStreamTablesResponse) ProtoReflect() protoreflect.Message { + mi := &file_binlogdata_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VStreamTablesResponse.ProtoReflect.Descriptor instead. +func (*VStreamTablesResponse) Descriptor() ([]byte, []int) { + return file_binlogdata_proto_rawDescGZIP(), []int{25} +} + +func (x *VStreamTablesResponse) GetTableName() string { + if x != nil { + return x.TableName + } + return "" +} + +func (x *VStreamTablesResponse) GetFields() []*query.Field { + if x != nil { + return x.Fields + } + return nil +} + +func (x *VStreamTablesResponse) GetPkfields() []*query.Field { + if x != nil { + return x.Pkfields + } + return nil +} + +func (x *VStreamTablesResponse) GetGtid() string { + if x != nil { + return x.Gtid + } + return "" +} + +func (x *VStreamTablesResponse) GetRows() []*query.Row { + if x != nil { + return x.Rows + } + return nil +} + +func (x *VStreamTablesResponse) GetLastpk() *query.Row { + if x != nil { + return x.Lastpk + } + return nil +} + type LastPKEvent struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2446,7 +2609,7 @@ type LastPKEvent struct { func (x *LastPKEvent) Reset() { *x = LastPKEvent{} if protoimpl.UnsafeEnabled { - mi := &file_binlogdata_proto_msgTypes[24] + mi := &file_binlogdata_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2459,7 +2622,7 @@ func (x *LastPKEvent) String() string { func (*LastPKEvent) ProtoMessage() {} func (x *LastPKEvent) ProtoReflect() protoreflect.Message { - mi := &file_binlogdata_proto_msgTypes[24] + mi := &file_binlogdata_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2472,7 +2635,7 @@ func (x *LastPKEvent) ProtoReflect() protoreflect.Message { // Deprecated: Use LastPKEvent.ProtoReflect.Descriptor instead. func (*LastPKEvent) Descriptor() ([]byte, []int) { - return file_binlogdata_proto_rawDescGZIP(), []int{24} + return file_binlogdata_proto_rawDescGZIP(), []int{26} } func (x *LastPKEvent) GetTableLastPK() *TableLastPK { @@ -2501,7 +2664,7 @@ type TableLastPK struct { func (x *TableLastPK) Reset() { *x = TableLastPK{} if protoimpl.UnsafeEnabled { - mi := &file_binlogdata_proto_msgTypes[25] + mi := &file_binlogdata_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2514,7 +2677,7 @@ func (x *TableLastPK) String() string { func (*TableLastPK) ProtoMessage() {} func (x *TableLastPK) ProtoReflect() protoreflect.Message { - mi := &file_binlogdata_proto_msgTypes[25] + mi := &file_binlogdata_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2527,7 +2690,7 @@ func (x *TableLastPK) ProtoReflect() protoreflect.Message { // Deprecated: Use TableLastPK.ProtoReflect.Descriptor instead. func (*TableLastPK) Descriptor() ([]byte, []int) { - return file_binlogdata_proto_rawDescGZIP(), []int{25} + return file_binlogdata_proto_rawDescGZIP(), []int{27} } func (x *TableLastPK) GetTableName() string { @@ -2561,7 +2724,7 @@ type VStreamResultsRequest struct { func (x *VStreamResultsRequest) Reset() { *x = VStreamResultsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_binlogdata_proto_msgTypes[26] + mi := &file_binlogdata_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2574,7 +2737,7 @@ func (x *VStreamResultsRequest) String() string { func (*VStreamResultsRequest) ProtoMessage() {} func (x *VStreamResultsRequest) ProtoReflect() protoreflect.Message { - mi := &file_binlogdata_proto_msgTypes[26] + mi := &file_binlogdata_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2587,7 +2750,7 @@ func (x *VStreamResultsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VStreamResultsRequest.ProtoReflect.Descriptor instead. func (*VStreamResultsRequest) Descriptor() ([]byte, []int) { - return file_binlogdata_proto_rawDescGZIP(), []int{26} + return file_binlogdata_proto_rawDescGZIP(), []int{28} } func (x *VStreamResultsRequest) GetEffectiveCallerId() *vtrpc.CallerID { @@ -2633,7 +2796,7 @@ type VStreamResultsResponse struct { func (x *VStreamResultsResponse) Reset() { *x = VStreamResultsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_binlogdata_proto_msgTypes[27] + mi := &file_binlogdata_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2646,7 +2809,7 @@ func (x *VStreamResultsResponse) String() string { func (*VStreamResultsResponse) ProtoMessage() {} func (x *VStreamResultsResponse) ProtoReflect() protoreflect.Message { - mi := &file_binlogdata_proto_msgTypes[27] + mi := &file_binlogdata_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2659,7 +2822,7 @@ func (x *VStreamResultsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VStreamResultsResponse.ProtoReflect.Descriptor instead. func (*VStreamResultsResponse) Descriptor() ([]byte, []int) { - return file_binlogdata_proto_rawDescGZIP(), []int{27} + return file_binlogdata_proto_rawDescGZIP(), []int{29} } func (x *VStreamResultsResponse) GetFields() []*query.Field { @@ -2699,7 +2862,7 @@ type BinlogTransaction_Statement struct { func (x *BinlogTransaction_Statement) Reset() { *x = BinlogTransaction_Statement{} if protoimpl.UnsafeEnabled { - mi := &file_binlogdata_proto_msgTypes[28] + mi := &file_binlogdata_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2712,7 +2875,7 @@ func (x *BinlogTransaction_Statement) String() string { func (*BinlogTransaction_Statement) ProtoMessage() {} func (x *BinlogTransaction_Statement) ProtoReflect() protoreflect.Message { - mi := &file_binlogdata_proto_msgTypes[28] + mi := &file_binlogdata_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2761,7 +2924,7 @@ type RowChange_Bitmap struct { func (x *RowChange_Bitmap) Reset() { *x = RowChange_Bitmap{} if protoimpl.UnsafeEnabled { - mi := &file_binlogdata_proto_msgTypes[32] + mi := &file_binlogdata_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2774,7 +2937,7 @@ func (x *RowChange_Bitmap) String() string { func (*RowChange_Bitmap) ProtoMessage() {} func (x *RowChange_Bitmap) ProtoReflect() protoreflect.Message { - mi := &file_binlogdata_proto_msgTypes[32] + mi := &file_binlogdata_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2987,7 +3150,7 @@ var file_binlogdata_proto_rawDesc = []byte{ 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x32, 0x0a, 0x06, 0x42, 0x69, 0x74, 0x6d, 0x61, 0x70, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, - 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x22, 0x93, + 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x22, 0xa9, 0x01, 0x0a, 0x08, 0x52, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x0b, 0x72, 0x6f, @@ -2997,166 +3160,155 @@ var file_binlogdata_proto_rawDesc = []byte{ 0x65, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x22, 0x83, 0x01, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x22, 0x83, 0x01, 0x0a, 0x0a, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1a, + 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x22, 0x88, 0x01, 0x0a, 0x09, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x12, 0x1a, + 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x67, 0x74, 0x69, 0x64, 0x12, 0x35, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x5f, + 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, + 0x4b, 0x52, 0x08, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x4b, 0x73, 0x22, 0x3f, 0x0a, 0x05, 0x56, + 0x47, 0x74, 0x69, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x67, 0x74, + 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, + 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x73, 0x22, 0x41, 0x0a, 0x0d, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, + 0xbc, 0x02, 0x0a, 0x07, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x40, 0x0a, 0x0e, 0x6d, + 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, + 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, + 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, + 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x0b, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x47, + 0x74, 0x69, 0x64, 0x73, 0x12, 0x3d, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, + 0x61, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, + 0x6e, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x8b, + 0x04, 0x0a, 0x06, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x31, 0x0a, 0x09, 0x72, 0x6f, 0x77, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x08, + 0x72, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x0b, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, + 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x12, 0x27, 0x0a, 0x05, 0x76, 0x67, 0x74, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x11, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x47, + 0x74, 0x69, 0x64, 0x52, 0x05, 0x76, 0x67, 0x74, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x07, 0x6a, 0x6f, + 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x62, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, + 0x52, 0x07, 0x6a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x6d, 0x6c, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64, 0x6d, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x63, + 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0b, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x3c, + 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, + 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x18, 0x17, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1c, + 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x68, 0x0a, 0x0c, + 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x5f, 0x6b, 0x5f, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x70, 0x4b, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x41, 0x0a, 0x0d, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, + 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x30, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0xc7, 0x02, 0x0a, 0x0e, 0x56, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, + 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, + 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, + 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, + 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, + 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, + 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x12, 0x3e, 0x0a, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, + 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, + 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, + 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, + 0x50, 0x4b, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x22, 0x85, 0x02, 0x0a, 0x12, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, + 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, + 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, + 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, + 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, + 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, + 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, + 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x2a, + 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0xf9, 0x01, 0x0a, 0x13, 0x56, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, - 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x88, 0x01, 0x0a, 0x09, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, - 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x35, - 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x08, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x50, 0x4b, 0x73, 0x22, 0x3f, 0x0a, 0x05, 0x56, 0x47, 0x74, 0x69, 0x64, 0x12, 0x36, - 0x0a, 0x0b, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x47, 0x74, 0x69, 0x64, 0x73, 0x22, 0x41, 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xbc, 0x02, 0x0a, 0x07, 0x4a, 0x6f, - 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x40, 0x0a, 0x0e, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, - 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x67, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, - 0x25, 0x0a, 0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x6f, - 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x0b, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, - 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69, - 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, - 0x69, 0x64, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x73, 0x12, 0x3d, - 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e, 0x74, 0x73, 0x18, 0x06, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e, 0x74, 0x73, 0x12, 0x29, 0x0a, - 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x8b, 0x04, 0x0a, 0x06, 0x56, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x16, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, - 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x12, 0x0a, - 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, - 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, - 0x31, 0x0a, 0x09, 0x72, 0x6f, 0x77, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x52, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x72, 0x6f, 0x77, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, - 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x27, 0x0a, 0x05, 0x76, - 0x67, 0x74, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x62, 0x69, 0x6e, - 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x47, 0x74, 0x69, 0x64, 0x52, 0x05, 0x76, - 0x67, 0x74, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x07, 0x6a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x07, 0x6a, 0x6f, 0x75, 0x72, - 0x6e, 0x61, 0x6c, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x6d, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x64, 0x6d, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, - 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x63, 0x75, 0x72, - 0x72, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, - 0x5f, 0x70, 0x5f, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x61, - 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x50, - 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x17, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f, - 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, - 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x68, 0x0a, 0x0c, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, - 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, + 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, - 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x5f, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x70, 0x4b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x22, 0x41, 0x0a, 0x0d, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x12, 0x30, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, - 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x06, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x22, 0xc7, 0x02, 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, - 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, - 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, - 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, - 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, - 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, - 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, - 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, - 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x2a, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, - 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x3e, 0x0a, - 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x73, - 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, - 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x73, 0x22, 0x3d, 0x0a, - 0x0f, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x2a, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x85, 0x02, 0x0a, - 0x12, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, - 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, - 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, - 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, - 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, - 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, - 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, - 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, - 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, - 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, - 0x73, 0x74, 0x70, 0x6b, 0x22, 0xf9, 0x01, 0x0a, 0x13, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, - 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, - 0x64, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, - 0x6c, 0x64, 0x52, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, - 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, - 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, - 0x12, 0x22, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, - 0x73, 0x74, 0x70, 0x6b, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, - 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, - 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, - 0x22, 0x69, 0x0a, 0x0b, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, - 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, - 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1c, 0x0a, - 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22, 0x58, 0x0a, 0x0b, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, - 0x74, 0x70, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, - 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0xdc, 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, + 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, + 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, + 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, + 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, + 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, + 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x68, 0x65, 0x61, + 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x22, 0xc5, 0x01, 0x0a, 0x14, 0x56, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, @@ -3167,61 +3319,101 @@ var file_binlogdata_proto_rawDesc = []byte{ 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, - 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x22, 0x72, 0x0a, 0x16, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, - 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, - 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, - 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x2a, 0x3e, 0x0a, 0x0b, 0x4f, 0x6e, 0x44, 0x44, - 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x47, 0x4e, 0x4f, 0x52, - 0x45, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x54, 0x4f, 0x50, 0x10, 0x01, 0x12, 0x08, 0x0a, - 0x04, 0x45, 0x58, 0x45, 0x43, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x58, 0x45, 0x43, 0x5f, - 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x03, 0x2a, 0x7b, 0x0a, 0x18, 0x56, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, - 0x69, 0x7a, 0x65, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4c, - 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, - 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x65, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, - 0x44, 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x34, 0x0a, 0x1b, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, - 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, - 0x0a, 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x2a, 0x71, 0x0a, 0x19, 0x56, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, - 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12, - 0x0b, 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, - 0x43, 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, - 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, - 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x2a, 0x8d, - 0x02, 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, - 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x54, - 0x49, 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49, 0x4e, 0x10, 0x02, 0x12, - 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x52, - 0x4f, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x44, 0x4c, - 0x10, 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x06, 0x12, 0x0b, - 0x0a, 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12, 0x0a, 0x0a, 0x06, 0x55, - 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, - 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, - 0x4f, 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0c, - 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x48, - 0x45, 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x47, - 0x54, 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55, 0x52, 0x4e, 0x41, 0x4c, - 0x10, 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, - 0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12, 0x0d, 0x0a, 0x09, 0x53, - 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, - 0x50, 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x14, 0x2a, 0x27, - 0x0a, 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x0a, 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, - 0x48, 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76, 0x69, 0x74, 0x65, 0x73, - 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, - 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, - 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xde, + 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a, + 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70, + 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, + 0x6f, 0x77, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c, + 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, + 0x69, 0x0a, 0x0b, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x3c, + 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, + 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1c, 0x0a, 0x09, + 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22, 0x58, 0x0a, 0x0b, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, + 0x70, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, + 0x73, 0x74, 0x70, 0x6b, 0x22, 0xdc, 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, + 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, + 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, + 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, + 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, + 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, + 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, + 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, + 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x22, 0x72, 0x0a, 0x16, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, + 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, + 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, + 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x2a, 0x3e, 0x0a, 0x0b, 0x4f, 0x6e, 0x44, 0x44, 0x4c, + 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, + 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x54, 0x4f, 0x50, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, + 0x45, 0x58, 0x45, 0x43, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x58, 0x45, 0x43, 0x5f, 0x49, + 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x03, 0x2a, 0x7b, 0x0a, 0x18, 0x56, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4c, 0x6f, + 0x6f, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x4d, + 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x65, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x44, + 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x44, 0x0a, 0x1b, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, + 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x74, + 0x6f, 0x6d, 0x69, 0x63, 0x43, 0x6f, 0x70, 0x79, 0x10, 0x02, 0x2a, 0x71, 0x0a, 0x19, 0x56, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, + 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12, 0x0b, + 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x43, + 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x6e, + 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x05, + 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x2a, 0x8d, 0x02, + 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, + 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x54, 0x49, + 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49, 0x4e, 0x10, 0x02, 0x12, 0x0a, + 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x4f, + 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x44, 0x4c, 0x10, + 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x06, 0x12, 0x0b, 0x0a, + 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, + 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, + 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x4f, + 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0c, 0x12, + 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x45, + 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x47, 0x54, + 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55, 0x52, 0x4e, 0x41, 0x4c, 0x10, + 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, 0x0a, + 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x41, + 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, 0x50, + 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x14, 0x2a, 0x27, 0x0a, + 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, + 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x48, + 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, + 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, + 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -3237,7 +3429,7 @@ func file_binlogdata_proto_rawDescGZIP() []byte { } var file_binlogdata_proto_enumTypes = make([]protoimpl.EnumInfo, 8) -var file_binlogdata_proto_msgTypes = make([]protoimpl.MessageInfo, 33) +var file_binlogdata_proto_msgTypes = make([]protoimpl.MessageInfo, 35) var file_binlogdata_proto_goTypes = []interface{}{ (OnDDLAction)(0), // 0: binlogdata.OnDDLAction (VReplicationWorkflowType)(0), // 1: binlogdata.VReplicationWorkflowType @@ -3271,48 +3463,50 @@ var file_binlogdata_proto_goTypes = []interface{}{ (*VStreamResponse)(nil), // 29: binlogdata.VStreamResponse (*VStreamRowsRequest)(nil), // 30: binlogdata.VStreamRowsRequest (*VStreamRowsResponse)(nil), // 31: binlogdata.VStreamRowsResponse - (*LastPKEvent)(nil), // 32: binlogdata.LastPKEvent - (*TableLastPK)(nil), // 33: binlogdata.TableLastPK - (*VStreamResultsRequest)(nil), // 34: binlogdata.VStreamResultsRequest - (*VStreamResultsResponse)(nil), // 35: binlogdata.VStreamResultsResponse - (*BinlogTransaction_Statement)(nil), // 36: binlogdata.BinlogTransaction.Statement - nil, // 37: binlogdata.Rule.ConvertEnumToTextEntry - nil, // 38: binlogdata.Rule.ConvertCharsetEntry - nil, // 39: binlogdata.Rule.ConvertIntToEnumEntry - (*RowChange_Bitmap)(nil), // 40: binlogdata.RowChange.Bitmap - (*query.EventToken)(nil), // 41: query.EventToken - (*topodata.KeyRange)(nil), // 42: topodata.KeyRange - (topodata.TabletType)(0), // 43: topodata.TabletType - (*query.Row)(nil), // 44: query.Row - (*query.Field)(nil), // 45: query.Field - (*vtrpc.CallerID)(nil), // 46: vtrpc.CallerID - (*query.VTGateCallerID)(nil), // 47: query.VTGateCallerID - (*query.Target)(nil), // 48: query.Target - (*query.QueryResult)(nil), // 49: query.QueryResult + (*VStreamTablesRequest)(nil), // 32: binlogdata.VStreamTablesRequest + (*VStreamTablesResponse)(nil), // 33: binlogdata.VStreamTablesResponse + (*LastPKEvent)(nil), // 34: binlogdata.LastPKEvent + (*TableLastPK)(nil), // 35: binlogdata.TableLastPK + (*VStreamResultsRequest)(nil), // 36: binlogdata.VStreamResultsRequest + (*VStreamResultsResponse)(nil), // 37: binlogdata.VStreamResultsResponse + (*BinlogTransaction_Statement)(nil), // 38: binlogdata.BinlogTransaction.Statement + nil, // 39: binlogdata.Rule.ConvertEnumToTextEntry + nil, // 40: binlogdata.Rule.ConvertCharsetEntry + nil, // 41: binlogdata.Rule.ConvertIntToEnumEntry + (*RowChange_Bitmap)(nil), // 42: binlogdata.RowChange.Bitmap + (*query.EventToken)(nil), // 43: query.EventToken + (*topodata.KeyRange)(nil), // 44: topodata.KeyRange + (topodata.TabletType)(0), // 45: topodata.TabletType + (*query.Row)(nil), // 46: query.Row + (*query.Field)(nil), // 47: query.Field + (*vtrpc.CallerID)(nil), // 48: vtrpc.CallerID + (*query.VTGateCallerID)(nil), // 49: query.VTGateCallerID + (*query.Target)(nil), // 50: query.Target + (*query.QueryResult)(nil), // 51: query.QueryResult } var file_binlogdata_proto_depIdxs = []int32{ - 36, // 0: binlogdata.BinlogTransaction.statements:type_name -> binlogdata.BinlogTransaction.Statement - 41, // 1: binlogdata.BinlogTransaction.event_token:type_name -> query.EventToken - 42, // 2: binlogdata.StreamKeyRangeRequest.key_range:type_name -> topodata.KeyRange + 38, // 0: binlogdata.BinlogTransaction.statements:type_name -> binlogdata.BinlogTransaction.Statement + 43, // 1: binlogdata.BinlogTransaction.event_token:type_name -> query.EventToken + 44, // 2: binlogdata.StreamKeyRangeRequest.key_range:type_name -> topodata.KeyRange 8, // 3: binlogdata.StreamKeyRangeRequest.charset:type_name -> binlogdata.Charset 9, // 4: binlogdata.StreamKeyRangeResponse.binlog_transaction:type_name -> binlogdata.BinlogTransaction 8, // 5: binlogdata.StreamTablesRequest.charset:type_name -> binlogdata.Charset 9, // 6: binlogdata.StreamTablesResponse.binlog_transaction:type_name -> binlogdata.BinlogTransaction - 37, // 7: binlogdata.Rule.convert_enum_to_text:type_name -> binlogdata.Rule.ConvertEnumToTextEntry - 38, // 8: binlogdata.Rule.convert_charset:type_name -> binlogdata.Rule.ConvertCharsetEntry - 39, // 9: binlogdata.Rule.convert_int_to_enum:type_name -> binlogdata.Rule.ConvertIntToEnumEntry + 39, // 7: binlogdata.Rule.convert_enum_to_text:type_name -> binlogdata.Rule.ConvertEnumToTextEntry + 40, // 8: binlogdata.Rule.convert_charset:type_name -> binlogdata.Rule.ConvertCharsetEntry + 41, // 9: binlogdata.Rule.convert_int_to_enum:type_name -> binlogdata.Rule.ConvertIntToEnumEntry 15, // 10: binlogdata.Filter.rules:type_name -> binlogdata.Rule 7, // 11: binlogdata.Filter.field_event_mode:type_name -> binlogdata.Filter.FieldEventMode - 43, // 12: binlogdata.BinlogSource.tablet_type:type_name -> topodata.TabletType - 42, // 13: binlogdata.BinlogSource.key_range:type_name -> topodata.KeyRange + 45, // 12: binlogdata.BinlogSource.tablet_type:type_name -> topodata.TabletType + 44, // 13: binlogdata.BinlogSource.key_range:type_name -> topodata.KeyRange 16, // 14: binlogdata.BinlogSource.filter:type_name -> binlogdata.Filter 0, // 15: binlogdata.BinlogSource.on_ddl:type_name -> binlogdata.OnDDLAction - 44, // 16: binlogdata.RowChange.before:type_name -> query.Row - 44, // 17: binlogdata.RowChange.after:type_name -> query.Row - 40, // 18: binlogdata.RowChange.data_columns:type_name -> binlogdata.RowChange.Bitmap + 46, // 16: binlogdata.RowChange.before:type_name -> query.Row + 46, // 17: binlogdata.RowChange.after:type_name -> query.Row + 42, // 18: binlogdata.RowChange.data_columns:type_name -> binlogdata.RowChange.Bitmap 18, // 19: binlogdata.RowEvent.row_changes:type_name -> binlogdata.RowChange - 45, // 20: binlogdata.FieldEvent.fields:type_name -> query.Field - 33, // 21: binlogdata.ShardGtid.table_p_ks:type_name -> binlogdata.TableLastPK + 47, // 20: binlogdata.FieldEvent.fields:type_name -> query.Field + 35, // 21: binlogdata.ShardGtid.table_p_ks:type_name -> binlogdata.TableLastPK 21, // 22: binlogdata.VGtid.shard_gtids:type_name -> binlogdata.ShardGtid 5, // 23: binlogdata.Journal.migration_type:type_name -> binlogdata.MigrationType 21, // 24: binlogdata.Journal.shard_gtids:type_name -> binlogdata.ShardGtid @@ -3322,38 +3516,45 @@ var file_binlogdata_proto_depIdxs = []int32{ 20, // 28: binlogdata.VEvent.field_event:type_name -> binlogdata.FieldEvent 22, // 29: binlogdata.VEvent.vgtid:type_name -> binlogdata.VGtid 24, // 30: binlogdata.VEvent.journal:type_name -> binlogdata.Journal - 32, // 31: binlogdata.VEvent.last_p_k_event:type_name -> binlogdata.LastPKEvent - 45, // 32: binlogdata.MinimalTable.fields:type_name -> query.Field + 34, // 31: binlogdata.VEvent.last_p_k_event:type_name -> binlogdata.LastPKEvent + 47, // 32: binlogdata.MinimalTable.fields:type_name -> query.Field 26, // 33: binlogdata.MinimalSchema.tables:type_name -> binlogdata.MinimalTable - 46, // 34: binlogdata.VStreamRequest.effective_caller_id:type_name -> vtrpc.CallerID - 47, // 35: binlogdata.VStreamRequest.immediate_caller_id:type_name -> query.VTGateCallerID - 48, // 36: binlogdata.VStreamRequest.target:type_name -> query.Target + 48, // 34: binlogdata.VStreamRequest.effective_caller_id:type_name -> vtrpc.CallerID + 49, // 35: binlogdata.VStreamRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 50, // 36: binlogdata.VStreamRequest.target:type_name -> query.Target 16, // 37: binlogdata.VStreamRequest.filter:type_name -> binlogdata.Filter - 33, // 38: binlogdata.VStreamRequest.table_last_p_ks:type_name -> binlogdata.TableLastPK + 35, // 38: binlogdata.VStreamRequest.table_last_p_ks:type_name -> binlogdata.TableLastPK 25, // 39: binlogdata.VStreamResponse.events:type_name -> binlogdata.VEvent - 46, // 40: binlogdata.VStreamRowsRequest.effective_caller_id:type_name -> vtrpc.CallerID - 47, // 41: binlogdata.VStreamRowsRequest.immediate_caller_id:type_name -> query.VTGateCallerID - 48, // 42: binlogdata.VStreamRowsRequest.target:type_name -> query.Target - 49, // 43: binlogdata.VStreamRowsRequest.lastpk:type_name -> query.QueryResult - 45, // 44: binlogdata.VStreamRowsResponse.fields:type_name -> query.Field - 45, // 45: binlogdata.VStreamRowsResponse.pkfields:type_name -> query.Field - 44, // 46: binlogdata.VStreamRowsResponse.rows:type_name -> query.Row - 44, // 47: binlogdata.VStreamRowsResponse.lastpk:type_name -> query.Row - 33, // 48: binlogdata.LastPKEvent.table_last_p_k:type_name -> binlogdata.TableLastPK - 49, // 49: binlogdata.TableLastPK.lastpk:type_name -> query.QueryResult - 46, // 50: binlogdata.VStreamResultsRequest.effective_caller_id:type_name -> vtrpc.CallerID - 47, // 51: binlogdata.VStreamResultsRequest.immediate_caller_id:type_name -> query.VTGateCallerID - 48, // 52: binlogdata.VStreamResultsRequest.target:type_name -> query.Target - 45, // 53: binlogdata.VStreamResultsResponse.fields:type_name -> query.Field - 44, // 54: binlogdata.VStreamResultsResponse.rows:type_name -> query.Row - 6, // 55: binlogdata.BinlogTransaction.Statement.category:type_name -> binlogdata.BinlogTransaction.Statement.Category - 8, // 56: binlogdata.BinlogTransaction.Statement.charset:type_name -> binlogdata.Charset - 14, // 57: binlogdata.Rule.ConvertCharsetEntry.value:type_name -> binlogdata.CharsetConversion - 58, // [58:58] is the sub-list for method output_type - 58, // [58:58] is the sub-list for method input_type - 58, // [58:58] is the sub-list for extension type_name - 58, // [58:58] is the sub-list for extension extendee - 0, // [0:58] is the sub-list for field type_name + 48, // 40: binlogdata.VStreamRowsRequest.effective_caller_id:type_name -> vtrpc.CallerID + 49, // 41: binlogdata.VStreamRowsRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 50, // 42: binlogdata.VStreamRowsRequest.target:type_name -> query.Target + 51, // 43: binlogdata.VStreamRowsRequest.lastpk:type_name -> query.QueryResult + 47, // 44: binlogdata.VStreamRowsResponse.fields:type_name -> query.Field + 47, // 45: binlogdata.VStreamRowsResponse.pkfields:type_name -> query.Field + 46, // 46: binlogdata.VStreamRowsResponse.rows:type_name -> query.Row + 46, // 47: binlogdata.VStreamRowsResponse.lastpk:type_name -> query.Row + 48, // 48: binlogdata.VStreamTablesRequest.effective_caller_id:type_name -> vtrpc.CallerID + 49, // 49: binlogdata.VStreamTablesRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 50, // 50: binlogdata.VStreamTablesRequest.target:type_name -> query.Target + 47, // 51: binlogdata.VStreamTablesResponse.fields:type_name -> query.Field + 47, // 52: binlogdata.VStreamTablesResponse.pkfields:type_name -> query.Field + 46, // 53: binlogdata.VStreamTablesResponse.rows:type_name -> query.Row + 46, // 54: binlogdata.VStreamTablesResponse.lastpk:type_name -> query.Row + 35, // 55: binlogdata.LastPKEvent.table_last_p_k:type_name -> binlogdata.TableLastPK + 51, // 56: binlogdata.TableLastPK.lastpk:type_name -> query.QueryResult + 48, // 57: binlogdata.VStreamResultsRequest.effective_caller_id:type_name -> vtrpc.CallerID + 49, // 58: binlogdata.VStreamResultsRequest.immediate_caller_id:type_name -> query.VTGateCallerID + 50, // 59: binlogdata.VStreamResultsRequest.target:type_name -> query.Target + 47, // 60: binlogdata.VStreamResultsResponse.fields:type_name -> query.Field + 46, // 61: binlogdata.VStreamResultsResponse.rows:type_name -> query.Row + 6, // 62: binlogdata.BinlogTransaction.Statement.category:type_name -> binlogdata.BinlogTransaction.Statement.Category + 8, // 63: binlogdata.BinlogTransaction.Statement.charset:type_name -> binlogdata.Charset + 14, // 64: binlogdata.Rule.ConvertCharsetEntry.value:type_name -> binlogdata.CharsetConversion + 65, // [65:65] is the sub-list for method output_type + 65, // [65:65] is the sub-list for method input_type + 65, // [65:65] is the sub-list for extension type_name + 65, // [65:65] is the sub-list for extension extendee + 0, // [0:65] is the sub-list for field type_name } func init() { file_binlogdata_proto_init() } @@ -3651,7 +3852,7 @@ func file_binlogdata_proto_init() { } } file_binlogdata_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LastPKEvent); i { + switch v := v.(*VStreamTablesRequest); i { case 0: return &v.state case 1: @@ -3663,7 +3864,7 @@ func file_binlogdata_proto_init() { } } file_binlogdata_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableLastPK); i { + switch v := v.(*VStreamTablesResponse); i { case 0: return &v.state case 1: @@ -3675,7 +3876,7 @@ func file_binlogdata_proto_init() { } } file_binlogdata_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VStreamResultsRequest); i { + switch v := v.(*LastPKEvent); i { case 0: return &v.state case 1: @@ -3687,7 +3888,7 @@ func file_binlogdata_proto_init() { } } file_binlogdata_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VStreamResultsResponse); i { + switch v := v.(*TableLastPK); i { case 0: return &v.state case 1: @@ -3699,6 +3900,30 @@ func file_binlogdata_proto_init() { } } file_binlogdata_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VStreamResultsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_binlogdata_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*VStreamResultsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_binlogdata_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BinlogTransaction_Statement); i { case 0: return &v.state @@ -3710,7 +3935,7 @@ func file_binlogdata_proto_init() { return nil } } - file_binlogdata_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + file_binlogdata_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RowChange_Bitmap); i { case 0: return &v.state @@ -3729,7 +3954,7 @@ func file_binlogdata_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_binlogdata_proto_rawDesc, NumEnums: 8, - NumMessages: 33, + NumMessages: 35, NumExtensions: 0, NumServices: 0, }, diff --git a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go index 78c7faa0e17..c3b86502979 100644 --- a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go +++ b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go @@ -1660,6 +1660,11 @@ func (m *RowEvent) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.Flags != 0 { + i = encodeVarint(dAtA, i, uint64(m.Flags)) + i-- + dAtA[i] = 0x28 + } if len(m.Shard) > 0 { i -= len(m.Shard) copy(dAtA[i:], m.Shard) @@ -2606,6 +2611,162 @@ func (m *VStreamRowsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *VStreamTablesRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VStreamTablesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VStreamTablesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Target != nil { + size, err := m.Target.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + if m.ImmediateCallerId != nil { + size, err := m.ImmediateCallerId.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.EffectiveCallerId != nil { + size, err := m.EffectiveCallerId.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *VStreamTablesResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VStreamTablesResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VStreamTablesResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Lastpk != nil { + size, err := m.Lastpk.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if len(m.Rows) > 0 { + for iNdEx := len(m.Rows) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Rows[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x2a + } + } + if len(m.Gtid) > 0 { + i -= len(m.Gtid) + copy(dAtA[i:], m.Gtid) + i = encodeVarint(dAtA, i, uint64(len(m.Gtid))) + i-- + dAtA[i] = 0x22 + } + if len(m.Pkfields) > 0 { + for iNdEx := len(m.Pkfields) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Pkfields[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Fields) > 0 { + for iNdEx := len(m.Fields) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Fields[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + } + if len(m.TableName) > 0 { + i -= len(m.TableName) + copy(dAtA[i:], m.TableName) + i = encodeVarint(dAtA, i, uint64(len(m.TableName))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *LastPKEvent) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -2889,6 +3050,29 @@ func (m *VStreamRowsResponse) ReturnToVTPool() { func VStreamRowsResponseFromVTPool() *VStreamRowsResponse { return vtprotoPool_VStreamRowsResponse.Get().(*VStreamRowsResponse) } + +var vtprotoPool_VStreamTablesResponse = sync.Pool{ + New: func() interface{} { + return &VStreamTablesResponse{} + }, +} + +func (m *VStreamTablesResponse) ResetVT() { + for _, mm := range m.Rows { + mm.ResetVT() + } + m.Lastpk.ReturnToVTPool() + m.Reset() +} +func (m *VStreamTablesResponse) ReturnToVTPool() { + if m != nil { + m.ResetVT() + vtprotoPool_VStreamTablesResponse.Put(m) + } +} +func VStreamTablesResponseFromVTPool() *VStreamTablesResponse { + return vtprotoPool_VStreamTablesResponse.Get().(*VStreamTablesResponse) +} func (m *Charset) SizeVT() (n int) { if m == nil { return 0 @@ -3246,6 +3430,9 @@ func (m *RowEvent) SizeVT() (n int) { if l > 0 { n += 1 + l + sov(uint64(l)) } + if m.Flags != 0 { + n += 1 + sov(uint64(m.Flags)) + } n += len(m.unknownFields) return n } @@ -3613,6 +3800,68 @@ func (m *VStreamRowsResponse) SizeVT() (n int) { return n } +func (m *VStreamTablesRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.EffectiveCallerId != nil { + l = m.EffectiveCallerId.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.ImmediateCallerId != nil { + l = m.ImmediateCallerId.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.Target != nil { + l = m.Target.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *VStreamTablesResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.TableName) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Fields) > 0 { + for _, e := range m.Fields { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + if len(m.Pkfields) > 0 { + for _, e := range m.Pkfields { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + l = len(m.Gtid) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if len(m.Rows) > 0 { + for _, e := range m.Rows { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + if m.Lastpk != nil { + l = m.Lastpk.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + func (m *LastPKEvent) SizeVT() (n int) { if m == nil { return 0 @@ -6234,6 +6483,25 @@ func (m *RowEvent) UnmarshalVT(dAtA []byte) error { } m.Shard = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Flags", wireType) + } + m.Flags = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Flags |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -8672,6 +8940,439 @@ func (m *VStreamRowsResponse) UnmarshalVT(dAtA []byte) error { } return nil } +func (m *VStreamTablesRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VStreamTablesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VStreamTablesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EffectiveCallerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EffectiveCallerId == nil { + m.EffectiveCallerId = &vtrpc.CallerID{} + } + if err := m.EffectiveCallerId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ImmediateCallerId", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ImmediateCallerId == nil { + m.ImmediateCallerId = &query.VTGateCallerID{} + } + if err := m.ImmediateCallerId.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &query.Target{} + } + if err := m.Target.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *VStreamTablesResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VStreamTablesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VStreamTablesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TableName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TableName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Fields", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if len(m.Fields) == cap(m.Fields) { + m.Fields = append(m.Fields, &query.Field{}) + } else { + m.Fields = m.Fields[:len(m.Fields)+1] + if m.Fields[len(m.Fields)-1] == nil { + m.Fields[len(m.Fields)-1] = &query.Field{} + } + } + if err := m.Fields[len(m.Fields)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pkfields", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if len(m.Pkfields) == cap(m.Pkfields) { + m.Pkfields = append(m.Pkfields, &query.Field{}) + } else { + m.Pkfields = m.Pkfields[:len(m.Pkfields)+1] + if m.Pkfields[len(m.Pkfields)-1] == nil { + m.Pkfields[len(m.Pkfields)-1] = &query.Field{} + } + } + if err := m.Pkfields[len(m.Pkfields)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Gtid", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Gtid = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Rows", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if len(m.Rows) == cap(m.Rows) { + m.Rows = append(m.Rows, &query.Row{}) + } else { + m.Rows = m.Rows[:len(m.Rows)+1] + if m.Rows[len(m.Rows)-1] == nil { + m.Rows[len(m.Rows)-1] = &query.Row{} + } + } + if err := m.Rows[len(m.Rows)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Lastpk", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Lastpk == nil { + m.Lastpk = query.RowFromVTPool() + } + if err := m.Lastpk.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *LastPKEvent) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/go/vt/proto/queryservice/queryservice.pb.go b/go/vt/proto/queryservice/queryservice.pb.go index 7213f3322f2..babedcde966 100644 --- a/go/vt/proto/queryservice/queryservice.pb.go +++ b/go/vt/proto/queryservice/queryservice.pb.go @@ -45,7 +45,7 @@ var file_queryservice_proto_rawDesc = []byte{ 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x1a, 0x0b, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x10, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x32, 0xd2, 0x10, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x07, 0x45, + 0x6f, 0x32, 0xac, 0x11, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x07, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x12, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, @@ -168,20 +168,26 @@ var file_queryservice_proto_rawDesc = []byte{ 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x5b, 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x62, 0x69, 0x6e, - 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x30, 0x01, 0x12, 0x42, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, - 0x17, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, 0x2b, 0x5a, 0x29, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, - 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, - 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x73, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x58, 0x0a, 0x0d, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x20, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x5b, 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x12, 0x21, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x42, 0x0a, 0x09, + 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x17, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, + 0x42, 0x2b, 0x5a, 0x29, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, + 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var file_queryservice_proto_goTypes = []interface{}{ @@ -210,35 +216,37 @@ var file_queryservice_proto_goTypes = []interface{}{ (*query.StreamHealthRequest)(nil), // 22: query.StreamHealthRequest (*binlogdata.VStreamRequest)(nil), // 23: binlogdata.VStreamRequest (*binlogdata.VStreamRowsRequest)(nil), // 24: binlogdata.VStreamRowsRequest - (*binlogdata.VStreamResultsRequest)(nil), // 25: binlogdata.VStreamResultsRequest - (*query.GetSchemaRequest)(nil), // 26: query.GetSchemaRequest - (*query.ExecuteResponse)(nil), // 27: query.ExecuteResponse - (*query.StreamExecuteResponse)(nil), // 28: query.StreamExecuteResponse - (*query.BeginResponse)(nil), // 29: query.BeginResponse - (*query.CommitResponse)(nil), // 30: query.CommitResponse - (*query.RollbackResponse)(nil), // 31: query.RollbackResponse - (*query.PrepareResponse)(nil), // 32: query.PrepareResponse - (*query.CommitPreparedResponse)(nil), // 33: query.CommitPreparedResponse - (*query.RollbackPreparedResponse)(nil), // 34: query.RollbackPreparedResponse - (*query.CreateTransactionResponse)(nil), // 35: query.CreateTransactionResponse - (*query.StartCommitResponse)(nil), // 36: query.StartCommitResponse - (*query.SetRollbackResponse)(nil), // 37: query.SetRollbackResponse - (*query.ConcludeTransactionResponse)(nil), // 38: query.ConcludeTransactionResponse - (*query.ReadTransactionResponse)(nil), // 39: query.ReadTransactionResponse - (*query.BeginExecuteResponse)(nil), // 40: query.BeginExecuteResponse - (*query.BeginStreamExecuteResponse)(nil), // 41: query.BeginStreamExecuteResponse - (*query.MessageStreamResponse)(nil), // 42: query.MessageStreamResponse - (*query.MessageAckResponse)(nil), // 43: query.MessageAckResponse - (*query.ReserveExecuteResponse)(nil), // 44: query.ReserveExecuteResponse - (*query.ReserveBeginExecuteResponse)(nil), // 45: query.ReserveBeginExecuteResponse - (*query.ReserveStreamExecuteResponse)(nil), // 46: query.ReserveStreamExecuteResponse - (*query.ReserveBeginStreamExecuteResponse)(nil), // 47: query.ReserveBeginStreamExecuteResponse - (*query.ReleaseResponse)(nil), // 48: query.ReleaseResponse - (*query.StreamHealthResponse)(nil), // 49: query.StreamHealthResponse - (*binlogdata.VStreamResponse)(nil), // 50: binlogdata.VStreamResponse - (*binlogdata.VStreamRowsResponse)(nil), // 51: binlogdata.VStreamRowsResponse - (*binlogdata.VStreamResultsResponse)(nil), // 52: binlogdata.VStreamResultsResponse - (*query.GetSchemaResponse)(nil), // 53: query.GetSchemaResponse + (*binlogdata.VStreamTablesRequest)(nil), // 25: binlogdata.VStreamTablesRequest + (*binlogdata.VStreamResultsRequest)(nil), // 26: binlogdata.VStreamResultsRequest + (*query.GetSchemaRequest)(nil), // 27: query.GetSchemaRequest + (*query.ExecuteResponse)(nil), // 28: query.ExecuteResponse + (*query.StreamExecuteResponse)(nil), // 29: query.StreamExecuteResponse + (*query.BeginResponse)(nil), // 30: query.BeginResponse + (*query.CommitResponse)(nil), // 31: query.CommitResponse + (*query.RollbackResponse)(nil), // 32: query.RollbackResponse + (*query.PrepareResponse)(nil), // 33: query.PrepareResponse + (*query.CommitPreparedResponse)(nil), // 34: query.CommitPreparedResponse + (*query.RollbackPreparedResponse)(nil), // 35: query.RollbackPreparedResponse + (*query.CreateTransactionResponse)(nil), // 36: query.CreateTransactionResponse + (*query.StartCommitResponse)(nil), // 37: query.StartCommitResponse + (*query.SetRollbackResponse)(nil), // 38: query.SetRollbackResponse + (*query.ConcludeTransactionResponse)(nil), // 39: query.ConcludeTransactionResponse + (*query.ReadTransactionResponse)(nil), // 40: query.ReadTransactionResponse + (*query.BeginExecuteResponse)(nil), // 41: query.BeginExecuteResponse + (*query.BeginStreamExecuteResponse)(nil), // 42: query.BeginStreamExecuteResponse + (*query.MessageStreamResponse)(nil), // 43: query.MessageStreamResponse + (*query.MessageAckResponse)(nil), // 44: query.MessageAckResponse + (*query.ReserveExecuteResponse)(nil), // 45: query.ReserveExecuteResponse + (*query.ReserveBeginExecuteResponse)(nil), // 46: query.ReserveBeginExecuteResponse + (*query.ReserveStreamExecuteResponse)(nil), // 47: query.ReserveStreamExecuteResponse + (*query.ReserveBeginStreamExecuteResponse)(nil), // 48: query.ReserveBeginStreamExecuteResponse + (*query.ReleaseResponse)(nil), // 49: query.ReleaseResponse + (*query.StreamHealthResponse)(nil), // 50: query.StreamHealthResponse + (*binlogdata.VStreamResponse)(nil), // 51: binlogdata.VStreamResponse + (*binlogdata.VStreamRowsResponse)(nil), // 52: binlogdata.VStreamRowsResponse + (*binlogdata.VStreamTablesResponse)(nil), // 53: binlogdata.VStreamTablesResponse + (*binlogdata.VStreamResultsResponse)(nil), // 54: binlogdata.VStreamResultsResponse + (*query.GetSchemaResponse)(nil), // 55: query.GetSchemaResponse } var file_queryservice_proto_depIdxs = []int32{ 0, // 0: queryservice.Query.Execute:input_type -> query.ExecuteRequest @@ -266,37 +274,39 @@ var file_queryservice_proto_depIdxs = []int32{ 22, // 22: queryservice.Query.StreamHealth:input_type -> query.StreamHealthRequest 23, // 23: queryservice.Query.VStream:input_type -> binlogdata.VStreamRequest 24, // 24: queryservice.Query.VStreamRows:input_type -> binlogdata.VStreamRowsRequest - 25, // 25: queryservice.Query.VStreamResults:input_type -> binlogdata.VStreamResultsRequest - 26, // 26: queryservice.Query.GetSchema:input_type -> query.GetSchemaRequest - 27, // 27: queryservice.Query.Execute:output_type -> query.ExecuteResponse - 28, // 28: queryservice.Query.StreamExecute:output_type -> query.StreamExecuteResponse - 29, // 29: queryservice.Query.Begin:output_type -> query.BeginResponse - 30, // 30: queryservice.Query.Commit:output_type -> query.CommitResponse - 31, // 31: queryservice.Query.Rollback:output_type -> query.RollbackResponse - 32, // 32: queryservice.Query.Prepare:output_type -> query.PrepareResponse - 33, // 33: queryservice.Query.CommitPrepared:output_type -> query.CommitPreparedResponse - 34, // 34: queryservice.Query.RollbackPrepared:output_type -> query.RollbackPreparedResponse - 35, // 35: queryservice.Query.CreateTransaction:output_type -> query.CreateTransactionResponse - 36, // 36: queryservice.Query.StartCommit:output_type -> query.StartCommitResponse - 37, // 37: queryservice.Query.SetRollback:output_type -> query.SetRollbackResponse - 38, // 38: queryservice.Query.ConcludeTransaction:output_type -> query.ConcludeTransactionResponse - 39, // 39: queryservice.Query.ReadTransaction:output_type -> query.ReadTransactionResponse - 40, // 40: queryservice.Query.BeginExecute:output_type -> query.BeginExecuteResponse - 41, // 41: queryservice.Query.BeginStreamExecute:output_type -> query.BeginStreamExecuteResponse - 42, // 42: queryservice.Query.MessageStream:output_type -> query.MessageStreamResponse - 43, // 43: queryservice.Query.MessageAck:output_type -> query.MessageAckResponse - 44, // 44: queryservice.Query.ReserveExecute:output_type -> query.ReserveExecuteResponse - 45, // 45: queryservice.Query.ReserveBeginExecute:output_type -> query.ReserveBeginExecuteResponse - 46, // 46: queryservice.Query.ReserveStreamExecute:output_type -> query.ReserveStreamExecuteResponse - 47, // 47: queryservice.Query.ReserveBeginStreamExecute:output_type -> query.ReserveBeginStreamExecuteResponse - 48, // 48: queryservice.Query.Release:output_type -> query.ReleaseResponse - 49, // 49: queryservice.Query.StreamHealth:output_type -> query.StreamHealthResponse - 50, // 50: queryservice.Query.VStream:output_type -> binlogdata.VStreamResponse - 51, // 51: queryservice.Query.VStreamRows:output_type -> binlogdata.VStreamRowsResponse - 52, // 52: queryservice.Query.VStreamResults:output_type -> binlogdata.VStreamResultsResponse - 53, // 53: queryservice.Query.GetSchema:output_type -> query.GetSchemaResponse - 27, // [27:54] is the sub-list for method output_type - 0, // [0:27] is the sub-list for method input_type + 25, // 25: queryservice.Query.VStreamTables:input_type -> binlogdata.VStreamTablesRequest + 26, // 26: queryservice.Query.VStreamResults:input_type -> binlogdata.VStreamResultsRequest + 27, // 27: queryservice.Query.GetSchema:input_type -> query.GetSchemaRequest + 28, // 28: queryservice.Query.Execute:output_type -> query.ExecuteResponse + 29, // 29: queryservice.Query.StreamExecute:output_type -> query.StreamExecuteResponse + 30, // 30: queryservice.Query.Begin:output_type -> query.BeginResponse + 31, // 31: queryservice.Query.Commit:output_type -> query.CommitResponse + 32, // 32: queryservice.Query.Rollback:output_type -> query.RollbackResponse + 33, // 33: queryservice.Query.Prepare:output_type -> query.PrepareResponse + 34, // 34: queryservice.Query.CommitPrepared:output_type -> query.CommitPreparedResponse + 35, // 35: queryservice.Query.RollbackPrepared:output_type -> query.RollbackPreparedResponse + 36, // 36: queryservice.Query.CreateTransaction:output_type -> query.CreateTransactionResponse + 37, // 37: queryservice.Query.StartCommit:output_type -> query.StartCommitResponse + 38, // 38: queryservice.Query.SetRollback:output_type -> query.SetRollbackResponse + 39, // 39: queryservice.Query.ConcludeTransaction:output_type -> query.ConcludeTransactionResponse + 40, // 40: queryservice.Query.ReadTransaction:output_type -> query.ReadTransactionResponse + 41, // 41: queryservice.Query.BeginExecute:output_type -> query.BeginExecuteResponse + 42, // 42: queryservice.Query.BeginStreamExecute:output_type -> query.BeginStreamExecuteResponse + 43, // 43: queryservice.Query.MessageStream:output_type -> query.MessageStreamResponse + 44, // 44: queryservice.Query.MessageAck:output_type -> query.MessageAckResponse + 45, // 45: queryservice.Query.ReserveExecute:output_type -> query.ReserveExecuteResponse + 46, // 46: queryservice.Query.ReserveBeginExecute:output_type -> query.ReserveBeginExecuteResponse + 47, // 47: queryservice.Query.ReserveStreamExecute:output_type -> query.ReserveStreamExecuteResponse + 48, // 48: queryservice.Query.ReserveBeginStreamExecute:output_type -> query.ReserveBeginStreamExecuteResponse + 49, // 49: queryservice.Query.Release:output_type -> query.ReleaseResponse + 50, // 50: queryservice.Query.StreamHealth:output_type -> query.StreamHealthResponse + 51, // 51: queryservice.Query.VStream:output_type -> binlogdata.VStreamResponse + 52, // 52: queryservice.Query.VStreamRows:output_type -> binlogdata.VStreamRowsResponse + 53, // 53: queryservice.Query.VStreamTables:output_type -> binlogdata.VStreamTablesResponse + 54, // 54: queryservice.Query.VStreamResults:output_type -> binlogdata.VStreamResultsResponse + 55, // 55: queryservice.Query.GetSchema:output_type -> query.GetSchemaResponse + 28, // [28:56] is the sub-list for method output_type + 0, // [0:28] is the sub-list for method input_type 0, // [0:0] is the sub-list for extension type_name 0, // [0:0] is the sub-list for extension extendee 0, // [0:0] is the sub-list for field type_name diff --git a/go/vt/proto/queryservice/queryservice_grpc.pb.go b/go/vt/proto/queryservice/queryservice_grpc.pb.go index f9d596351e2..c05ea4f83a3 100644 --- a/go/vt/proto/queryservice/queryservice_grpc.pb.go +++ b/go/vt/proto/queryservice/queryservice_grpc.pb.go @@ -79,6 +79,8 @@ type QueryClient interface { VStream(ctx context.Context, in *binlogdata.VStreamRequest, opts ...grpc.CallOption) (Query_VStreamClient, error) // VStreamRows streams rows from the specified starting point. VStreamRows(ctx context.Context, in *binlogdata.VStreamRowsRequest, opts ...grpc.CallOption) (Query_VStreamRowsClient, error) + // VStreamTables streams rows from the specified starting point. + VStreamTables(ctx context.Context, in *binlogdata.VStreamTablesRequest, opts ...grpc.CallOption) (Query_VStreamTablesClient, error) // VStreamResults streams results along with the gtid of the snapshot. VStreamResults(ctx context.Context, in *binlogdata.VStreamResultsRequest, opts ...grpc.CallOption) (Query_VStreamResultsClient, error) // GetSchema returns the schema information. @@ -502,8 +504,40 @@ func (x *queryVStreamRowsClient) Recv() (*binlogdata.VStreamRowsResponse, error) return m, nil } +func (c *queryClient) VStreamTables(ctx context.Context, in *binlogdata.VStreamTablesRequest, opts ...grpc.CallOption) (Query_VStreamTablesClient, error) { + stream, err := c.cc.NewStream(ctx, &Query_ServiceDesc.Streams[8], "/queryservice.Query/VStreamTables", opts...) + if err != nil { + return nil, err + } + x := &queryVStreamTablesClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Query_VStreamTablesClient interface { + Recv() (*binlogdata.VStreamTablesResponse, error) + grpc.ClientStream +} + +type queryVStreamTablesClient struct { + grpc.ClientStream +} + +func (x *queryVStreamTablesClient) Recv() (*binlogdata.VStreamTablesResponse, error) { + m := new(binlogdata.VStreamTablesResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + func (c *queryClient) VStreamResults(ctx context.Context, in *binlogdata.VStreamResultsRequest, opts ...grpc.CallOption) (Query_VStreamResultsClient, error) { - stream, err := c.cc.NewStream(ctx, &Query_ServiceDesc.Streams[8], "/queryservice.Query/VStreamResults", opts...) + stream, err := c.cc.NewStream(ctx, &Query_ServiceDesc.Streams[9], "/queryservice.Query/VStreamResults", opts...) if err != nil { return nil, err } @@ -535,7 +569,7 @@ func (x *queryVStreamResultsClient) Recv() (*binlogdata.VStreamResultsResponse, } func (c *queryClient) GetSchema(ctx context.Context, in *query.GetSchemaRequest, opts ...grpc.CallOption) (Query_GetSchemaClient, error) { - stream, err := c.cc.NewStream(ctx, &Query_ServiceDesc.Streams[9], "/queryservice.Query/GetSchema", opts...) + stream, err := c.cc.NewStream(ctx, &Query_ServiceDesc.Streams[10], "/queryservice.Query/GetSchema", opts...) if err != nil { return nil, err } @@ -625,6 +659,8 @@ type QueryServer interface { VStream(*binlogdata.VStreamRequest, Query_VStreamServer) error // VStreamRows streams rows from the specified starting point. VStreamRows(*binlogdata.VStreamRowsRequest, Query_VStreamRowsServer) error + // VStreamTables streams rows from the specified starting point. + VStreamTables(*binlogdata.VStreamTablesRequest, Query_VStreamTablesServer) error // VStreamResults streams results along with the gtid of the snapshot. VStreamResults(*binlogdata.VStreamResultsRequest, Query_VStreamResultsServer) error // GetSchema returns the schema information. @@ -711,6 +747,9 @@ func (UnimplementedQueryServer) VStream(*binlogdata.VStreamRequest, Query_VStrea func (UnimplementedQueryServer) VStreamRows(*binlogdata.VStreamRowsRequest, Query_VStreamRowsServer) error { return status.Errorf(codes.Unimplemented, "method VStreamRows not implemented") } +func (UnimplementedQueryServer) VStreamTables(*binlogdata.VStreamTablesRequest, Query_VStreamTablesServer) error { + return status.Errorf(codes.Unimplemented, "method VStreamTables not implemented") +} func (UnimplementedQueryServer) VStreamResults(*binlogdata.VStreamResultsRequest, Query_VStreamResultsServer) error { return status.Errorf(codes.Unimplemented, "method VStreamResults not implemented") } @@ -1204,6 +1243,27 @@ func (x *queryVStreamRowsServer) Send(m *binlogdata.VStreamRowsResponse) error { return x.ServerStream.SendMsg(m) } +func _Query_VStreamTables_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(binlogdata.VStreamTablesRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QueryServer).VStreamTables(m, &queryVStreamTablesServer{stream}) +} + +type Query_VStreamTablesServer interface { + Send(*binlogdata.VStreamTablesResponse) error + grpc.ServerStream +} + +type queryVStreamTablesServer struct { + grpc.ServerStream +} + +func (x *queryVStreamTablesServer) Send(m *binlogdata.VStreamTablesResponse) error { + return x.ServerStream.SendMsg(m) +} + func _Query_VStreamResults_Handler(srv interface{}, stream grpc.ServerStream) error { m := new(binlogdata.VStreamResultsRequest) if err := stream.RecvMsg(m); err != nil { @@ -1363,6 +1423,11 @@ var Query_ServiceDesc = grpc.ServiceDesc{ Handler: _Query_VStreamRows_Handler, ServerStreams: true, }, + { + StreamName: "VStreamTables", + Handler: _Query_VStreamTables_Handler, + ServerStreams: true, + }, { StreamName: "VStreamResults", Handler: _Query_VStreamResults_Handler, diff --git a/go/vt/proto/vtctldata/vtctldata.pb.go b/go/vt/proto/vtctldata/vtctldata.pb.go index e0942f1b3d9..03b4b2f7b0f 100644 --- a/go/vt/proto/vtctldata/vtctldata.pb.go +++ b/go/vt/proto/vtctldata/vtctldata.pb.go @@ -485,6 +485,7 @@ type MaterializeSettings struct { // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table copy finishes. DeferSecondaryKeys bool `protobuf:"varint,14,opt,name=defer_secondary_keys,json=deferSecondaryKeys,proto3" json:"defer_secondary_keys,omitempty"` TabletSelectionPreference tabletmanagerdata.TabletSelectionPreference `protobuf:"varint,15,opt,name=tablet_selection_preference,json=tabletSelectionPreference,proto3,enum=tabletmanagerdata.TabletSelectionPreference" json:"tablet_selection_preference,omitempty"` + AtomicCopy bool `protobuf:"varint,16,opt,name=atomic_copy,json=atomicCopy,proto3" json:"atomic_copy,omitempty"` } func (x *MaterializeSettings) Reset() { @@ -624,6 +625,13 @@ func (x *MaterializeSettings) GetTabletSelectionPreference() tabletmanagerdata.T return tabletmanagerdata.TabletSelectionPreference(0) } +func (x *MaterializeSettings) GetAtomicCopy() bool { + if x != nil { + return x.AtomicCopy + } + return false +} + type Keyspace struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7134,6 +7142,8 @@ type MoveTablesCreateRequest struct { AutoStart bool `protobuf:"varint,17,opt,name=auto_start,json=autoStart,proto3" json:"auto_start,omitempty"` // NoRoutingRules is set to true if routing rules should not be created on the target when the workflow is created. NoRoutingRules bool `protobuf:"varint,18,opt,name=no_routing_rules,json=noRoutingRules,proto3" json:"no_routing_rules,omitempty"` + // Run a single copy phase for the entire database. + AtomicCopy bool `protobuf:"varint,19,opt,name=atomic_copy,json=atomicCopy,proto3" json:"atomic_copy,omitempty"` } func (x *MoveTablesCreateRequest) Reset() { @@ -7294,6 +7304,13 @@ func (x *MoveTablesCreateRequest) GetNoRoutingRules() bool { return false } +func (x *MoveTablesCreateRequest) GetAtomicCopy() bool { + if x != nil { + return x.AtomicCopy + } + return false +} + type MoveTablesCreateResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -13496,8 +13513,8 @@ var file_vtctldata_proto_rawDesc = []byte{ 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x64, 0x6c, 0x22, 0xe2, - 0x05, 0x0a, 0x13, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x65, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x64, 0x6c, 0x22, 0x83, + 0x06, 0x0a, 0x13, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, @@ -13543,1613 +13560,1682 @@ var file_vtctldata_proto_rawDesc = []byte{ 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, - 0x6e, 0x63, 0x65, 0x22, 0x4e, 0x0a, 0x08, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, - 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x22, 0x85, 0x13, 0x0a, 0x0f, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, - 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x16, 0x0a, - 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x2f, 0x0a, 0x13, 0x6d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, - 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x3f, 0x0a, 0x08, - 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x23, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x72, 0x61, 0x74, - 0x65, 0x67, 0x79, 0x52, 0x08, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x18, 0x0a, - 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x27, 0x0a, 0x08, 0x61, 0x64, 0x64, 0x65, 0x64, - 0x5f, 0x61, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, - 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x07, 0x61, 0x64, 0x64, 0x65, 0x64, 0x41, 0x74, - 0x12, 0x2f, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, - 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x41, - 0x74, 0x12, 0x27, 0x0a, 0x08, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x61, 0x74, 0x18, 0x0b, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, - 0x65, 0x52, 0x07, 0x72, 0x65, 0x61, 0x64, 0x79, 0x41, 0x74, 0x12, 0x2b, 0x0a, 0x0a, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, - 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x3b, 0x0a, 0x12, 0x6c, 0x69, 0x76, 0x65, 0x6e, - 0x65, 0x73, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0d, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, - 0x65, 0x52, 0x11, 0x6c, 0x69, 0x76, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, - 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, - 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, - 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x30, 0x0a, 0x0d, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x65, 0x64, - 0x5f, 0x75, 0x70, 0x5f, 0x61, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, - 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x63, 0x6c, 0x65, 0x61, - 0x6e, 0x65, 0x64, 0x55, 0x70, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x11, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1c, 0x0a, - 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x72, - 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x13, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x72, 0x65, - 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, - 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x66, - 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x15, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, - 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, 0x70, - 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x69, 0x67, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x17, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, - 0x74, 0x65, 0x78, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x64, 0x6c, 0x5f, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x64, 0x6c, 0x41, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x19, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, 0x0a, - 0x0b, 0x65, 0x74, 0x61, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x1a, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0a, 0x65, 0x74, 0x61, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x1f, - 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x1b, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x1c, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2a, - 0x0a, 0x11, 0x61, 0x64, 0x64, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6b, - 0x65, 0x79, 0x73, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, 0x61, 0x64, 0x64, 0x65, 0x64, - 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x72, 0x65, - 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6b, 0x65, 0x79, - 0x73, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x64, - 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, - 0x67, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6c, 0x6f, - 0x67, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x3f, 0x0a, 0x12, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, - 0x74, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x20, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x74, - 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x13, 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, - 0x6e, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x21, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x12, 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, 0x6e, 0x65, 0x43, 0x6f, 0x6d, - 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x18, 0x72, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x73, 0x18, 0x22, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x72, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x64, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x73, - 0x12, 0x44, 0x0a, 0x1f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x6e, 0x6f, 0x5f, 0x64, - 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x73, 0x18, 0x23, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x64, 0x72, 0x6f, 0x70, 0x70, - 0x65, 0x64, 0x4e, 0x6f, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x64, - 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, - 0x24, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x64, 0x65, 0x64, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, - 0x76, 0x65, 0x72, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x74, 0x65, 0x73, 0x18, 0x25, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x76, 0x65, 0x72, 0x74, 0x69, 0x62, 0x6c, 0x65, - 0x4e, 0x6f, 0x74, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x63, - 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x26, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, - 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x75, 0x69, - 0x64, 0x18, 0x27, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x76, 0x65, 0x72, 0x74, 0x65, - 0x64, 0x55, 0x75, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x76, 0x69, 0x65, 0x77, - 0x18, 0x28, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, 0x56, 0x69, 0x65, 0x77, 0x12, 0x2a, - 0x0a, 0x11, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, - 0x65, 0x74, 0x65, 0x18, 0x29, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x61, 0x64, 0x79, - 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x3a, 0x0a, 0x19, 0x76, 0x69, - 0x74, 0x65, 0x73, 0x73, 0x5f, 0x6c, 0x69, 0x76, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x5f, 0x69, 0x6e, - 0x64, 0x69, 0x63, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x76, - 0x69, 0x74, 0x65, 0x73, 0x73, 0x4c, 0x69, 0x76, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x49, 0x6e, 0x64, - 0x69, 0x63, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x2e, 0x0a, 0x13, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x74, - 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x2b, 0x20, - 0x01, 0x28, 0x02, 0x52, 0x11, 0x75, 0x73, 0x65, 0x72, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, - 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x70, 0x65, 0x63, 0x69, 0x61, - 0x6c, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x70, - 0x65, 0x63, 0x69, 0x61, 0x6c, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x38, 0x0a, 0x11, 0x6c, 0x61, 0x73, - 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x2d, + 0x6e, 0x63, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x5f, 0x63, 0x6f, + 0x70, 0x79, 0x18, 0x10, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, + 0x43, 0x6f, 0x70, 0x79, 0x22, 0x4e, 0x0a, 0x08, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x22, 0x85, 0x13, 0x0a, 0x0f, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, + 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x16, + 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x2f, 0x0a, 0x13, + 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x6d, 0x69, 0x67, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x3f, 0x0a, + 0x08, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x72, 0x61, + 0x74, 0x65, 0x67, 0x79, 0x52, 0x08, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x18, + 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x27, 0x0a, 0x08, 0x61, 0x64, 0x64, 0x65, + 0x64, 0x5f, 0x61, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, + 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x07, 0x61, 0x64, 0x64, 0x65, 0x64, 0x41, + 0x74, 0x12, 0x2f, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x61, + 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, + 0x41, 0x74, 0x12, 0x27, 0x0a, 0x08, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x61, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, - 0x6d, 0x65, 0x52, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, - 0x64, 0x41, 0x74, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, - 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x2e, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, 0x0c, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, 0x65, - 0x64, 0x5f, 0x61, 0x74, 0x18, 0x2f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, - 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, - 0x6c, 0x65, 0x64, 0x41, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, 0x6e, - 0x65, 0x5f, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x18, 0x30, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, - 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, 0x6e, 0x65, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x12, 0x14, - 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x31, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, - 0x74, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x75, 0x74, 0x6f, 0x76, 0x65, 0x72, 0x5f, - 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, - 0x63, 0x75, 0x74, 0x6f, 0x76, 0x65, 0x72, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x12, - 0x34, 0x0a, 0x16, 0x69, 0x73, 0x5f, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, - 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x33, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x14, 0x69, 0x73, 0x49, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x0b, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, 0x65, - 0x64, 0x5f, 0x61, 0x74, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, - 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, - 0x65, 0x64, 0x41, 0x74, 0x12, 0x3d, 0x0a, 0x14, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x74, 0x6f, - 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x18, 0x35, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, - 0x52, 0x11, 0x72, 0x65, 0x61, 0x64, 0x79, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, - 0x65, 0x41, 0x74, 0x22, 0x53, 0x0a, 0x08, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, - 0x0a, 0x0a, 0x06, 0x56, 0x49, 0x54, 0x45, 0x53, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x4f, - 0x4e, 0x4c, 0x49, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x47, 0x48, 0x4f, 0x53, 0x54, - 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x54, 0x4f, 0x53, 0x43, 0x10, 0x02, 0x12, 0x0a, 0x0a, - 0x06, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x59, 0x53, - 0x51, 0x4c, 0x10, 0x04, 0x1a, 0x02, 0x10, 0x01, 0x22, 0x71, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, - 0x0d, 0x0a, 0x09, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0d, - 0x0a, 0x09, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x12, 0x0a, 0x0a, - 0x06, 0x51, 0x55, 0x45, 0x55, 0x45, 0x44, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x45, 0x41, - 0x44, 0x59, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, - 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x06, 0x12, - 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x07, 0x22, 0x5e, 0x0a, 0x05, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xd2, 0x0c, 0x0a, 0x08, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, 0x06, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x3f, 0x0a, - 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, + 0x6d, 0x65, 0x52, 0x07, 0x72, 0x65, 0x61, 0x64, 0x79, 0x41, 0x74, 0x12, 0x2b, 0x0a, 0x0a, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x3b, 0x0a, 0x12, 0x6c, 0x69, 0x76, 0x65, + 0x6e, 0x65, 0x73, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0d, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x52, 0x11, 0x6c, 0x69, 0x76, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, + 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x30, 0x0a, 0x0d, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x65, + 0x64, 0x5f, 0x75, 0x70, 0x5f, 0x61, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, + 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x63, 0x6c, 0x65, + 0x61, 0x6e, 0x65, 0x64, 0x55, 0x70, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, + 0x11, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1c, + 0x0a, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, + 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x13, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x72, + 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, + 0x66, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x18, 0x15, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x12, 0x1a, 0x0a, 0x08, + 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, + 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x69, 0x67, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x17, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x64, 0x6c, 0x5f, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x64, 0x6c, 0x41, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x19, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x65, 0x74, 0x61, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x1a, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0a, 0x65, 0x74, 0x61, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, + 0x1f, 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x1b, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, + 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x1c, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x12, + 0x2a, 0x0a, 0x11, 0x61, 0x64, 0x64, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, + 0x6b, 0x65, 0x79, 0x73, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, 0x61, 0x64, 0x64, 0x65, + 0x64, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x72, + 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6b, 0x65, + 0x79, 0x73, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, + 0x64, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6c, + 0x6f, 0x67, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6c, + 0x6f, 0x67, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x3f, 0x0a, 0x12, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, + 0x63, 0x74, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x20, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x11, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, + 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x13, 0x70, 0x6f, 0x73, 0x74, 0x70, + 0x6f, 0x6e, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x21, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, 0x6e, 0x65, 0x43, 0x6f, + 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x18, 0x72, 0x65, 0x6d, 0x6f, + 0x76, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x18, 0x22, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x72, 0x65, 0x6d, 0x6f, + 0x76, 0x65, 0x64, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4b, 0x65, 0x79, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x12, 0x44, 0x0a, 0x1f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x6e, 0x6f, 0x5f, + 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x73, 0x18, 0x23, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x64, 0x72, 0x6f, 0x70, + 0x70, 0x65, 0x64, 0x4e, 0x6f, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x15, 0x65, 0x78, 0x70, 0x61, 0x6e, + 0x64, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x18, 0x24, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x64, 0x65, 0x64, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72, + 0x65, 0x76, 0x65, 0x72, 0x74, 0x69, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x74, 0x65, 0x73, 0x18, + 0x25, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x76, 0x65, 0x72, 0x74, 0x69, 0x62, 0x6c, + 0x65, 0x4e, 0x6f, 0x74, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, + 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x26, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, + 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x75, 0x75, + 0x69, 0x64, 0x18, 0x27, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x76, 0x65, 0x72, 0x74, + 0x65, 0x64, 0x55, 0x75, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x76, 0x69, 0x65, + 0x77, 0x18, 0x28, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, 0x56, 0x69, 0x65, 0x77, 0x12, + 0x2a, 0x0a, 0x11, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x18, 0x29, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x61, 0x64, + 0x79, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x3a, 0x0a, 0x19, 0x76, + 0x69, 0x74, 0x65, 0x73, 0x73, 0x5f, 0x6c, 0x69, 0x76, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x5f, 0x69, + 0x6e, 0x64, 0x69, 0x63, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, + 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x4c, 0x69, 0x76, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x49, 0x6e, + 0x64, 0x69, 0x63, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x2e, 0x0a, 0x13, 0x75, 0x73, 0x65, 0x72, 0x5f, + 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x5f, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x18, 0x2b, + 0x20, 0x01, 0x28, 0x02, 0x52, 0x11, 0x75, 0x73, 0x65, 0x72, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, + 0x6c, 0x65, 0x52, 0x61, 0x74, 0x69, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x70, 0x65, 0x63, 0x69, + 0x61, 0x6c, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, + 0x70, 0x65, 0x63, 0x69, 0x61, 0x6c, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x38, 0x0a, 0x11, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, + 0x2d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x52, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, + 0x65, 0x64, 0x41, 0x74, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, + 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x2e, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x54, 0x68, 0x72, 0x6f, + 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, 0x0c, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x6c, + 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x2f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, + 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x63, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x6c, 0x65, 0x64, 0x41, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, + 0x6e, 0x65, 0x5f, 0x6c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x18, 0x30, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x70, 0x6f, 0x6e, 0x65, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x67, 0x65, 0x18, 0x31, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x74, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x75, 0x74, 0x6f, 0x76, 0x65, 0x72, + 0x5f, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0f, 0x63, 0x75, 0x74, 0x6f, 0x76, 0x65, 0x72, 0x41, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, + 0x12, 0x34, 0x0a, 0x16, 0x69, 0x73, 0x5f, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, + 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x33, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x14, 0x69, 0x73, 0x49, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x0b, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, + 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, + 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x76, 0x69, 0x65, + 0x77, 0x65, 0x64, 0x41, 0x74, 0x12, 0x3d, 0x0a, 0x14, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x74, + 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x61, 0x74, 0x18, 0x35, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x52, 0x11, 0x72, 0x65, 0x61, 0x64, 0x79, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x41, 0x74, 0x22, 0x53, 0x0a, 0x08, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, + 0x12, 0x0a, 0x0a, 0x06, 0x56, 0x49, 0x54, 0x45, 0x53, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, + 0x4f, 0x4e, 0x4c, 0x49, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x47, 0x48, 0x4f, 0x53, + 0x54, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x54, 0x4f, 0x53, 0x43, 0x10, 0x02, 0x12, 0x0a, + 0x0a, 0x06, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x59, + 0x53, 0x51, 0x4c, 0x10, 0x04, 0x1a, 0x02, 0x10, 0x01, 0x22, 0x71, 0x0a, 0x06, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, + 0x12, 0x0d, 0x0a, 0x09, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, + 0x0d, 0x0a, 0x09, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x12, 0x0a, + 0x0a, 0x06, 0x51, 0x55, 0x45, 0x55, 0x45, 0x44, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x45, + 0x41, 0x44, 0x59, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, + 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x06, + 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x07, 0x22, 0x5e, 0x0a, 0x05, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xd2, 0x0c, 0x0a, + 0x08, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3f, 0x0a, + 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x31, - 0x0a, 0x15, 0x6d, 0x61, 0x78, 0x5f, 0x76, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6d, - 0x61, 0x78, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, - 0x67, 0x12, 0x4a, 0x0a, 0x0d, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x0c, 0x73, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x23, 0x0a, - 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, - 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x73, - 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x1a, 0x60, - 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 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, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x1a, 0x49, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0xb9, 0x01, 0x0a, 0x0b, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x34, 0x0a, 0x07, 0x73, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x73, 0x12, 0x46, 0x0a, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x73, 0x5f, - 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x1a, 0xf6, 0x06, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, - 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, - 0x67, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, - 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, 0x6c, - 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x0c, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, - 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x70, 0x50, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x17, 0x0a, - 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, - 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x15, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, - 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, - 0x69, 0x6d, 0x65, 0x52, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, - 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x74, - 0x69, 0x6d, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, 0x74, 0x61, - 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x0a, 0x63, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x04, 0x6c, - 0x6f, 0x67, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x3f, + 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, + 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, + 0x31, 0x0a, 0x15, 0x6d, 0x61, 0x78, 0x5f, 0x76, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, + 0x6d, 0x61, 0x78, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, + 0x61, 0x67, 0x12, 0x4a, 0x0a, 0x0d, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x4c, 0x6f, 0x67, 0x52, 0x04, 0x6c, 0x6f, 0x67, 0x73, 0x12, - 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x67, 0x5f, 0x66, 0x65, 0x74, 0x63, 0x68, 0x5f, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x6f, 0x67, 0x46, 0x65, 0x74, - 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, - 0x0f, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x1a, 0x3a, 0x0a, 0x09, 0x43, - 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x17, - 0x0a, 0x07, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x6b, 0x1a, 0xe6, 0x01, 0x0a, 0x03, 0x4c, 0x6f, 0x67, 0x12, - 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, - 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, - 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2b, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x64, 0x5f, 0x61, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, - 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x64, 0x41, 0x74, 0x12, 0x2b, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, - 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, - 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, - 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x6f, - 0x75, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, - 0x22, 0x59, 0x0a, 0x12, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, - 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x15, 0x0a, 0x13, 0x41, - 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x40, 0x0a, 0x14, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, - 0x69, 0x61, 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, 0x12, 0x14, - 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, - 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9e, 0x01, - 0x0a, 0x18, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x0d, 0x72, 0x6f, - 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x15, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x52, 0x6f, 0x75, 0x74, - 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, - 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, - 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x6b, - 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x62, - 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x0c, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x1b, - 0x0a, 0x19, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x1d, - 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, - 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4a, 0x0a, - 0x13, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, - 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, - 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x11, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, - 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, - 0x70, 0x5f, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0b, 0x73, 0x6b, 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x23, 0x0a, 0x0d, - 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x43, 0x65, 0x6c, 0x6c, - 0x73, 0x22, 0x20, 0x0a, 0x1e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0xef, 0x02, 0x0a, 0x12, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x64, 0x6c, 0x5f, - 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x64, 0x64, 0x6c, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x1b, 0x0a, 0x09, 0x75, - 0x75, 0x69, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, - 0x75, 0x75, 0x69, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x69, 0x67, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x73, - 0x6b, 0x69, 0x70, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x6b, 0x69, 0x70, 0x50, 0x72, 0x65, 0x66, 0x6c, 0x69, 0x67, - 0x68, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, - 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, - 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, - 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0a, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x4a, - 0x04, 0x08, 0x02, 0x10, 0x03, 0x22, 0xe8, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, 0x0a, - 0x09, 0x75, 0x75, 0x69, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x08, 0x75, 0x75, 0x69, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x6c, 0x0a, 0x16, 0x72, 0x6f, - 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, - 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, 0x73, - 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, + 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0c, 0x73, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x23, + 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x1a, + 0x60, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 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, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0xc3, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x49, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, 0x65, 0x62, - 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x6b, 0x69, 0x70, - 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, - 0x75, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, - 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x22, 0x44, 0x0a, 0x14, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, - 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xe5, 0x01, 0x0a, - 0x0d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, - 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, - 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, - 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, - 0x30, 0x0a, 0x14, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x5f, 0x66, - 0x72, 0x6f, 0x6d, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x69, - 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x46, 0x72, 0x6f, 0x6d, 0x50, 0x6f, - 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, 0x61, 0x66, - 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, - 0x53, 0x61, 0x66, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x0e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0xb9, 0x01, 0x0a, + 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x34, 0x0a, 0x07, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x73, 0x12, 0x46, 0x0a, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6e, + 0x74, 0x72, 0x6f, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x69, 0x73, + 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x1a, 0xf6, 0x06, 0x0a, 0x06, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0d, 0x62, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x0c, 0x62, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x70, + 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x17, + 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x15, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, + 0x54, 0x69, 0x6d, 0x65, 0x52, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, + 0x6d, 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, + 0x74, 0x69, 0x6d, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0b, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x52, 0x0a, 0x63, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x04, + 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x4c, 0x6f, 0x67, 0x52, 0x04, 0x6c, 0x6f, 0x67, 0x73, + 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x6f, 0x67, 0x5f, 0x66, 0x65, 0x74, 0x63, 0x68, 0x5f, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x6f, 0x67, 0x46, 0x65, + 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, + 0x18, 0x0f, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x1a, 0x3a, 0x0a, 0x09, + 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, + 0x17, 0x0a, 0x07, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x6b, 0x1a, 0xe6, 0x01, 0x0a, 0x03, 0x4c, 0x6f, 0x67, + 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, + 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x49, 0x64, 0x12, 0x12, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2b, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, + 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x41, 0x74, 0x12, 0x2b, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, + 0x61, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, + 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, + 0x74, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x22, 0x59, 0x0a, 0x12, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 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, 0x12, 0x2f, 0x0a, 0x09, 0x63, + 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x15, 0x0a, 0x13, + 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x40, 0x0a, 0x14, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, + 0x6c, 0x69, 0x61, 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, 0x12, + 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, + 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, + 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9e, + 0x01, 0x0a, 0x18, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, + 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x0d, 0x72, + 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x52, 0x6f, 0x75, + 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x74, 0x69, + 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, + 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, + 0x6b, 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x0c, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x22, + 0x1b, 0x0a, 0x19, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, + 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb3, 0x01, 0x0a, + 0x1d, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, + 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4a, + 0x0a, 0x13, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, + 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, + 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x11, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, + 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, + 0x69, 0x70, 0x5f, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0b, 0x73, 0x6b, 0x69, 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x23, 0x0a, + 0x0d, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x43, 0x65, 0x6c, + 0x6c, 0x73, 0x22, 0x20, 0x0a, 0x1e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xef, 0x02, 0x0a, 0x12, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x64, 0x6c, + 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x64, 0x64, 0x6c, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x1b, 0x0a, 0x09, + 0x75, 0x75, 0x69, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x08, 0x75, 0x75, 0x69, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x11, 0x6d, 0x69, 0x67, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, + 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x6b, 0x69, 0x70, 0x50, 0x72, 0x65, 0x66, 0x6c, 0x69, + 0x67, 0x68, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, + 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, + 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, + 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x22, 0xe8, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, + 0x0a, 0x09, 0x75, 0x75, 0x69, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x08, 0x75, 0x75, 0x69, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x6c, 0x0a, 0x16, 0x72, + 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, + 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, + 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, + 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0xc3, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x72, 0x65, + 0x62, 0x75, 0x69, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x6b, 0x69, + 0x70, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, + 0x72, 0x75, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, + 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x22, 0x44, 0x0a, 0x14, 0x41, 0x70, 0x70, 0x6c, 0x79, + 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xe5, 0x01, + 0x0a, 0x0d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, + 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, + 0x12, 0x30, 0x0a, 0x14, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x5f, + 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, + 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x46, 0x72, 0x6f, 0x6d, 0x50, + 0x6f, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, 0x61, + 0x66, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, + 0x65, 0x53, 0x61, 0x66, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x0e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x12, 0x24, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0xe2, 0x01, 0x0a, 0x12, 0x42, + 0x61, 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x12, 0x24, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0xe2, 0x01, 0x0a, 0x12, 0x42, 0x61, - 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, - 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, - 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, - 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6f, - 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, 0x67, - 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, 0x61, 0x66, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x53, 0x61, 0x66, 0x65, 0x12, 0x30, 0x0a, 0x14, - 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x5f, 0x66, 0x72, 0x6f, 0x6d, - 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x69, 0x6e, 0x63, 0x72, - 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x46, 0x72, 0x6f, 0x6d, 0x50, 0x6f, 0x73, 0x22, 0x4e, - 0x0a, 0x1c, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, - 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0xdf, - 0x01, 0x0a, 0x1d, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x76, 0x0a, 0x16, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x41, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x61, 0x6e, - 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, - 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, 0x73, + 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, + 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, + 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, + 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, + 0x67, 0x72, 0x61, 0x64, 0x65, 0x5f, 0x73, 0x61, 0x66, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0b, 0x75, 0x70, 0x67, 0x72, 0x61, 0x64, 0x65, 0x53, 0x61, 0x66, 0x65, 0x12, 0x30, 0x0a, + 0x14, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x5f, 0x66, 0x72, 0x6f, + 0x6d, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x69, 0x6e, 0x63, + 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x46, 0x72, 0x6f, 0x6d, 0x50, 0x6f, 0x73, 0x22, + 0x4e, 0x0a, 0x1c, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, + 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, + 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, + 0xdf, 0x01, 0x0a, 0x1d, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x76, 0x0a, 0x16, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x41, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x9b, 0x01, 0x0a, 0x17, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2d, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x06, 0x64, - 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x22, 0xa6, - 0x01, 0x0a, 0x18, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, - 0x79, 0x70, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, 0x0d, 0x62, - 0x65, 0x66, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x52, 0x0c, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x12, 0x33, 0x0a, 0x0c, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x0b, 0x61, 0x66, 0x74, 0x65, - 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x1e, 0x0a, 0x0b, 0x77, 0x61, 0x73, 0x5f, 0x64, - 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x77, 0x61, - 0x73, 0x44, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x22, 0x4f, 0x0a, 0x1d, 0x43, 0x6c, 0x65, 0x61, 0x6e, - 0x75, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0xe1, 0x01, 0x0a, 0x1e, 0x43, 0x6c, 0x65, - 0x61, 0x6e, 0x75, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x77, 0x0a, 0x16, 0x72, - 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x76, 0x74, - 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, - 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, + 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, + 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x17, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, + 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2d, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x06, + 0x64, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x22, + 0xa6, 0x01, 0x0a, 0x18, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, 0x0d, + 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x0c, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x12, 0x33, 0x0a, 0x0c, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x0b, 0x61, 0x66, 0x74, + 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x1e, 0x0a, 0x0b, 0x77, 0x61, 0x73, 0x5f, + 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x77, + 0x61, 0x73, 0x44, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x22, 0x4f, 0x0a, 0x1d, 0x43, 0x6c, 0x65, 0x61, + 0x6e, 0x75, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0xe1, 0x01, 0x0a, 0x1e, 0x43, 0x6c, + 0x65, 0x61, 0x6e, 0x75, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x77, 0x0a, 0x16, + 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, + 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 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, - 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x50, 0x0a, 0x1e, + 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x50, 0x0a, + 0x1e, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, + 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, + 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, + 0xe3, 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x78, 0x0a, 0x16, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, + 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, - 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0xe3, - 0x01, 0x0a, 0x1f, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x78, 0x0a, 0x16, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, - 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, - 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, - 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, - 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, + 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, - 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x99, 0x03, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 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, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x61, 0x6c, 0x6c, 0x6f, - 0x77, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x0c, 0x73, 0x65, 0x72, - 0x76, 0x65, 0x64, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x1d, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x0b, - 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x73, 0x12, 0x2a, 0x0a, 0x04, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x62, 0x61, 0x73, 0x65, 0x5f, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x62, 0x61, 0x73, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x31, 0x0a, 0x0d, - 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, - 0x65, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, - 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x70, 0x6f, - 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x64, 0x75, 0x72, 0x61, - 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x26, 0x0a, 0x0f, - 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x62, - 0x4e, 0x61, 0x6d, 0x65, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, - 0x22, 0x49, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x8c, 0x01, 0x0a, 0x12, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, - 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, - 0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, - 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x6e, 0x63, - 0x6c, 0x75, 0x64, 0x65, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x22, 0xa0, 0x01, 0x0a, 0x13, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x30, 0x0a, 0x14, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x5f, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, - 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x41, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0x41, 0x0a, - 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, + 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, + 0x18, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, + 0x68, 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x99, 0x03, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 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, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x0c, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x64, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, + 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x73, 0x12, 0x2a, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x62, 0x61, 0x73, 0x65, + 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x62, 0x61, 0x73, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x31, 0x0a, + 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x69, 0x6d, 0x65, + 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x70, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x64, 0x75, 0x72, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x26, 0x0a, + 0x0f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, + 0x06, 0x22, 0x49, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x8c, 0x01, 0x0a, + 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, + 0x6f, 0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, + 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x6e, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x22, 0xa0, 0x01, 0x0a, 0x13, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x30, 0x0a, 0x14, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x41, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0x41, + 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 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, 0x12, 0x14, 0x0a, 0x05, 0x66, + 0x6f, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, + 0x65, 0x22, 0x18, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2d, 0x0a, 0x17, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 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, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, - 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, - 0x22, 0x18, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2d, 0x0a, 0x17, 0x44, 0x65, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x1a, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 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, 0x1a, 0x0a, 0x18, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x67, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, - 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, - 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x18, - 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x13, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x28, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, - 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, - 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x76, 0x65, 0x6e, - 0x5f, 0x69, 0x66, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0d, 0x65, 0x76, 0x65, 0x6e, 0x49, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, - 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2d, - 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, - 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x1a, 0x0a, - 0x18, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x79, 0x0a, 0x14, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, - 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, - 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x22, 0x17, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x81, 0x03, - 0x0a, 0x1d, 0x45, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x6e, 0x63, 0x79, 0x52, 0x65, 0x70, 0x61, 0x72, - 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x67, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6e, - 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x3e, 0x0a, 0x0f, 0x69, 0x67, 0x6e, - 0x6f, 0x72, 0x65, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0e, 0x69, 0x67, 0x6e, 0x6f, 0x72, - 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, - 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, - 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, - 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, - 0x3f, 0x0a, 0x1c, 0x70, 0x72, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x72, 0x6f, 0x73, 0x73, - 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x19, 0x70, 0x72, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x72, - 0x6f, 0x73, 0x73, 0x43, 0x65, 0x6c, 0x6c, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x2f, 0x0a, 0x14, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x61, 0x6c, 0x6c, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, - 0x77, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x41, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x73, 0x22, 0xbc, 0x01, 0x0a, 0x1e, 0x45, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x6e, 0x63, 0x79, 0x52, - 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, - 0x65, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, + 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, + 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, + 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, + 0x18, 0x0a, 0x16, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x13, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x28, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x72, + 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, + 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x76, 0x65, + 0x6e, 0x5f, 0x69, 0x66, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0d, 0x65, 0x76, 0x65, 0x6e, 0x49, 0x66, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, + 0x67, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x2d, 0x0a, 0x17, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, + 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x1a, + 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x79, 0x0a, 0x14, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, + 0x61, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x52, 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, + 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x17, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x81, + 0x03, 0x0a, 0x1d, 0x45, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x6e, 0x63, 0x79, 0x52, 0x65, 0x70, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, + 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x3e, 0x0a, 0x0f, 0x69, 0x67, + 0x6e, 0x6f, 0x72, 0x65, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0e, 0x69, 0x67, 0x6e, 0x6f, + 0x72, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, + 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, + 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, + 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, + 0x12, 0x3f, 0x0a, 0x1c, 0x70, 0x72, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x72, 0x6f, 0x73, + 0x73, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x19, 0x70, 0x72, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x43, + 0x72, 0x6f, 0x73, 0x73, 0x43, 0x65, 0x6c, 0x6c, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x2f, 0x0a, 0x14, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x61, 0x6c, + 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x11, 0x77, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x41, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x73, 0x22, 0xbc, 0x01, 0x0a, 0x1e, 0x45, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x6e, 0x63, 0x79, + 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x10, 0x70, 0x72, 0x6f, 0x6d, 0x6f, + 0x74, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, + 0x65, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, + 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x73, 0x22, 0xa0, 0x01, 0x0a, 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, + 0x63, 0x68, 0x41, 0x73, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, + 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x19, + 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x07, 0x6d, 0x61, 0x78, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x75, 0x73, 0x65, + 0x5f, 0x70, 0x6f, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x75, 0x73, 0x65, + 0x50, 0x6f, 0x6f, 0x6c, 0x22, 0x47, 0x0a, 0x19, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, + 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0xd3, 0x01, + 0x0a, 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, + 0x44, 0x42, 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, - 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, - 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x22, 0xa0, 0x01, 0x0a, 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, - 0x68, 0x41, 0x73, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, - 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x19, 0x0a, - 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x07, 0x6d, 0x61, 0x78, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x5f, - 0x70, 0x6f, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x75, 0x73, 0x65, 0x50, - 0x6f, 0x6f, 0x6c, 0x22, 0x47, 0x0a, 0x19, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, - 0x74, 0x63, 0x68, 0x41, 0x73, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, + 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x61, + 0x78, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, + 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x23, + 0x0a, 0x0d, 0x72, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x72, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x22, 0x47, 0x0a, 0x19, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, + 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0xd3, 0x01, 0x0a, - 0x18, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, - 0x42, 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, - 0x69, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, 0x78, - 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x61, 0x78, - 0x52, 0x6f, 0x77, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x64, - 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x73, 0x12, 0x23, 0x0a, - 0x0d, 0x72, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x72, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x22, 0x47, 0x0a, 0x19, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, - 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0xa5, 0x01, 0x0a, 0x12, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, - 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x55, 0x0a, 0x13, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x68, 0x6f, 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x5e, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, - 0x6f, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x0b, 0x68, 0x6f, - 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0a, 0x68, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x22, 0x3c, 0x0a, 0x1e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x1f, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x49, - 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0x4b, 0x0a, 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, 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, 0x26, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x22, 0x9e, 0x01, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, - 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, - 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x25, 0x0a, 0x0e, - 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x4c, 0x69, - 0x6d, 0x69, 0x74, 0x22, 0x44, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x62, 0x61, 0x63, - 0x6b, 0x75, 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x79, 0x73, - 0x71, 0x6c, 0x63, 0x74, 0x6c, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, - 0x52, 0x07, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x22, 0x28, 0x0a, 0x12, 0x47, 0x65, 0x74, - 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, - 0x65, 0x6c, 0x6c, 0x22, 0x46, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, - 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x19, 0x0a, 0x17, 0x47, - 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x30, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, - 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x22, 0xb6, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x49, - 0x0a, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, - 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x1a, 0x50, 0x0a, 0x0c, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2a, 0x0a, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x50, 0x0a, 0x14, 0x47, - 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0xa5, 0x01, 0x0a, + 0x12, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x4c, 0x0a, - 0x15, 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x15, 0x0a, 0x13, 0x47, - 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x22, 0x49, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x09, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x52, 0x09, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x22, 0x30, 0x0a, - 0x12, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, - 0x46, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x51, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x50, 0x65, - 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x55, 0x0a, + 0x13, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x68, 0x6f, 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x5e, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, + 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x0b, 0x68, + 0x6f, 0x6f, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x0a, 0x68, 0x6f, 0x6f, 0x6b, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x22, 0x3c, 0x0a, 0x1e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x1f, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, + 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x1a, 0x4b, 0x0a, 0x0b, 0x53, 0x68, 0x61, 0x72, 0x64, 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, 0x26, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x9e, 0x01, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, + 0x69, 0x6d, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x08, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x12, 0x25, 0x0a, + 0x0e, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x4c, + 0x69, 0x6d, 0x69, 0x74, 0x22, 0x44, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x62, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x79, + 0x73, 0x71, 0x6c, 0x63, 0x74, 0x6c, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x07, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x22, 0x28, 0x0a, 0x12, 0x47, 0x65, + 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x46, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, + 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x19, 0x0a, 0x17, + 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x30, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x43, 0x65, + 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x47, 0x65, 0x74, + 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0xb6, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x49, 0x0a, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x1a, 0x50, 0x0a, 0x0c, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2a, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x50, 0x0a, 0x14, + 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, + 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x4c, + 0x0a, 0x15, 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x15, 0x0a, 0x13, + 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x49, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x09, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x09, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x22, 0x30, + 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x22, 0x46, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x51, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x50, + 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x5a, 0x0a, 0x16, 0x47, + 0x65, 0x74, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, + 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x6d, + 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x52, 0x6f, + 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x22, 0x55, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, + 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3a, 0x0a, 0x0d, + 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x52, 0x6f, + 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x74, + 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0xb0, 0x02, 0x0a, 0x10, 0x47, 0x65, 0x74, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, + 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, + 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x56, 0x69, 0x65, 0x77, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x73, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x4f, 0x6e, 0x6c, 0x79, 0x12, + 0x2a, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, + 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4f, 0x6e, 0x6c, 0x79, 0x22, 0x50, 0x0a, 0x11, 0x47, + 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x3b, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x23, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x66, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xb8, 0x02, + 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x11, + 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, + 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x39, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x28, 0x0a, 0x06, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x2e, + 0x0a, 0x05, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4f, + 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x05, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x14, + 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x6c, + 0x69, 0x6d, 0x69, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x6b, 0x69, 0x70, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x04, 0x73, 0x6b, 0x69, 0x70, 0x22, 0x59, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3a, 0x0a, 0x0a, 0x6d, 0x69, 0x67, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, + 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x22, 0x4c, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, + 0x65, 0x22, 0x3a, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x1d, 0x0a, + 0x1b, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, + 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x6a, 0x0a, 0x1c, + 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, + 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4a, 0x0a, 0x13, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, + 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, + 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x11, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, + 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0x32, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, + 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xf3, 0x01, 0x0a, + 0x1b, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x05, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x69, 0x0a, 0x0a, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x45, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4e, 0x61, 0x6d, + 0x65, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x1a, 0x20, 0x0a, 0x08, 0x4e, 0x61, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, + 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x22, 0x4a, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xcc, + 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x72, + 0x76, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x34, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x1a, 0x56, 0x0a, 0x11, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2b, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf8, 0x02, + 0x0a, 0x1c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, + 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x65, 0x6e, + 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x65, 0x6e, 0x61, 0x62, + 0x6c, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, + 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, + 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x73, 0x65, + 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x74, 0x12, 0x2d, 0x0a, 0x13, 0x63, 0x68, 0x65, 0x63, 0x6b, + 0x5f, 0x61, 0x73, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x6c, 0x66, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x41, 0x73, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x53, 0x65, 0x6c, 0x66, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, + 0x61, 0x73, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x41, 0x73, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x3f, 0x0a, 0x0d, 0x74, 0x68, 0x72, 0x6f, 0x74, + 0x74, 0x6c, 0x65, 0x64, 0x5f, 0x61, 0x70, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, + 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0c, 0x74, 0x68, 0x72, 0x6f, + 0x74, 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x22, 0x1f, 0x0a, 0x1d, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2a, 0x0a, 0x14, 0x47, 0x65, 0x74, + 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x4e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, + 0x0a, 0x0c, 0x73, 0x72, 0x76, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, + 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0a, 0x73, 0x72, 0x76, 0x56, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x2d, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, + 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, + 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xc5, 0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x56, 0x0a, 0x0d, 0x73, 0x72, 0x76, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x72, 0x76, 0x56, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x1a, 0x53, 0x0a, 0x10, 0x53, 0x72, 0x76, 0x56, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 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, 0x29, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4c, 0x0a, 0x10, + 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x5a, 0x0a, 0x16, 0x47, 0x65, - 0x74, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x40, 0x0a, 0x0b, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x65, - 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x6d, 0x69, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, - 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x22, 0x55, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3a, 0x0a, 0x0d, 0x72, - 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x52, 0x6f, 0x75, - 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x74, 0x69, - 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0xb0, 0x02, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x3d, 0x0a, 0x11, 0x47, 0x65, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x28, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0xe8, 0x01, 0x0a, 0x11, 0x47, 0x65, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, + 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x12, + 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, + 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0d, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x35, 0x0a, + 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x22, 0x40, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x07, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x07, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0x2c, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, + 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x70, 0x61, 0x74, 0x68, 0x22, 0x46, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, + 0x6f, 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x2b, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, + 0x67, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x66, 0x0a, 0x0c, + 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x12, 0x0a, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x70, 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, + 0x64, 0x72, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, + 0x64, 0x72, 0x65, 0x6e, 0x22, 0x2f, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x4d, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x22, 0x2e, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x22, 0x42, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x07, 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x8b, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1f, 0x0a, 0x0b, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x1b, 0x0a, + 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x08, 0x6e, 0x61, 0x6d, 0x65, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x22, 0x49, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, + 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x22, 0xfb, 0x01, 0x0a, 0x17, 0x49, 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, + 0x52, 0x0a, 0x1a, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x6c, 0x65, 0x63, 0x74, + 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x17, 0x70, 0x72, 0x69, 0x6d, + 0x61, 0x72, 0x79, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, + 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, + 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, + 0x42, 0x0a, 0x18, 0x49, 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, + 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, + 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x22, 0x4e, 0x0a, 0x1c, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, + 0x75, 0x69, 0x64, 0x22, 0xdf, 0x01, 0x0a, 0x1d, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x16, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, + 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, + 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, + 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, + 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, + 0x18, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, + 0x68, 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xbb, 0x06, 0x0a, 0x17, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x27, 0x0a, + 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, + 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x6c, + 0x0a, 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, + 0x65, 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x07, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x32, + 0x0a, 0x15, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, + 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x65, + 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x5a, 0x6f, 0x6e, 0x65, 0x12, 0x15, 0x0a, 0x06, + 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x6e, + 0x44, 0x64, 0x6c, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x61, 0x66, 0x74, 0x65, + 0x72, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x74, + 0x6f, 0x70, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x2a, 0x0a, 0x11, 0x64, + 0x72, 0x6f, 0x70, 0x5f, 0x66, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x73, + 0x18, 0x0f, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x46, 0x6f, 0x72, 0x65, + 0x69, 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x64, 0x65, 0x66, 0x65, 0x72, + 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, + 0x10, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, 0x66, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, + 0x6f, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, + 0x75, 0x74, 0x6f, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x6e, 0x6f, 0x5f, 0x72, + 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x12, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0e, 0x6e, 0x6f, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, + 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x5f, 0x63, 0x6f, 0x70, + 0x79, 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x43, + 0x6f, 0x70, 0x79, 0x22, 0xd5, 0x01, 0x0a, 0x18, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x48, 0x0a, 0x07, 0x64, 0x65, + 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, + 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x55, 0x0a, 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x07, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x22, 0xe9, 0x01, 0x0a, 0x19, + 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, + 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1b, + 0x0a, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x08, 0x6b, 0x65, 0x65, 0x70, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x12, 0x6b, + 0x65, 0x65, 0x70, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, + 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, 0x65, 0x70, 0x52, 0x6f, 0x75, + 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6e, + 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0c, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x17, + 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x22, 0x5e, 0x0a, 0x1a, 0x4d, 0x6f, 0x76, 0x65, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, + 0x26, 0x0a, 0x0f, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x4d, 0x0a, 0x11, 0x50, 0x69, 0x6e, 0x67, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x25, - 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x5f, 0x76, 0x69, 0x65, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, - 0x63, 0x6c, 0x75, 0x64, 0x65, 0x56, 0x69, 0x65, 0x77, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, - 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x28, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x69, - 0x7a, 0x65, 0x73, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x73, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x2a, - 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6f, - 0x6e, 0x6c, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4f, 0x6e, 0x6c, 0x79, 0x22, 0x50, 0x0a, 0x11, 0x47, 0x65, - 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x3b, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x23, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xb8, 0x02, 0x0a, - 0x1a, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x11, 0x6d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x39, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x12, 0x28, 0x0a, 0x06, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x2e, 0x0a, - 0x05, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4f, 0x72, - 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x05, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x14, 0x0a, - 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x6c, 0x69, - 0x6d, 0x69, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x6b, 0x69, 0x70, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x04, 0x73, 0x6b, 0x69, 0x70, 0x22, 0x59, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3a, 0x0a, 0x0a, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x22, 0x4c, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, + 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x14, 0x0a, 0x12, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x89, 0x02, 0x0a, + 0x1b, 0x50, 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, + 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x0d, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x5f, + 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0c, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0xba, 0x01, 0x0a, 0x1c, 0x50, 0x6c, 0x61, + 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x10, 0x70, + 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0f, 0x70, 0x72, + 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, + 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, + 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x74, 0x0a, 0x1b, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, + 0x6c, 0x6c, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x22, 0x1e, 0x0a, 0x1c, 0x52, + 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, + 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x1a, 0x52, + 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, + 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, + 0x6c, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, + 0x1d, 0x0a, 0x1b, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4f, + 0x0a, 0x13, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, + 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, + 0x16, 0x0a, 0x14, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x64, 0x0a, 0x1a, 0x52, 0x65, 0x66, 0x72, 0x65, + 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, - 0x22, 0x3a, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x1d, 0x0a, 0x1b, - 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, - 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x6a, 0x0a, 0x1c, 0x47, - 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4a, 0x0a, 0x13, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, - 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, - 0x75, 0x6c, 0x65, 0x73, 0x52, 0x11, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, - 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0x32, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, 0x72, - 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xf3, 0x01, 0x0a, 0x1b, - 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x05, 0x6e, - 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x6e, - 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x69, 0x0a, 0x0a, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x45, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4e, 0x61, 0x6d, 0x65, - 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, - 0x20, 0x0a, 0x08, 0x4e, 0x61, 0x6d, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x6e, - 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x22, 0x4a, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0xcc, 0x01, - 0x0a, 0x17, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x59, 0x0a, 0x0d, 0x73, 0x72, 0x76, - 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x34, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, - 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x73, 0x1a, 0x56, 0x0a, 0x11, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2b, 0x0a, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf8, 0x02, 0x0a, - 0x1c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x83, 0x01, + 0x0a, 0x1b, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, + 0x12, 0x69, 0x73, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, + 0x65, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x73, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x17, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, 0x64, + 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x44, 0x65, 0x74, 0x61, + 0x69, 0x6c, 0x73, 0x22, 0x4f, 0x0a, 0x13, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x01, 0x0a, + 0x1b, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x61, 0x69, 0x74, + 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0c, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, + 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, + 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x6f, 0x6e, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, 0x46, 0x0a, 0x1c, 0x52, 0x65, 0x6c, 0x6f, + 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, + 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x22, 0xbc, 0x01, 0x0a, 0x18, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x65, 0x6e, 0x61, - 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x65, 0x6e, 0x61, 0x62, 0x6c, - 0x65, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, - 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, - 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x73, - 0x74, 0x6f, 0x6d, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x28, 0x0a, 0x10, - 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x73, 0x65, 0x74, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x53, 0x65, 0x74, 0x12, 0x2d, 0x0a, 0x13, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, - 0x61, 0x73, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x6c, 0x66, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x10, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x41, 0x73, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x53, 0x65, 0x6c, 0x66, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x61, - 0x73, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x11, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x41, 0x73, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x3f, 0x0a, 0x0d, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, - 0x6c, 0x65, 0x64, 0x5f, 0x61, 0x70, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, - 0x65, 0x64, 0x41, 0x70, 0x70, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0c, 0x74, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x64, 0x41, 0x70, 0x70, 0x22, 0x1f, 0x0a, 0x1d, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2a, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, - 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x4e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, - 0x0c, 0x73, 0x72, 0x76, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x72, - 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0a, 0x73, 0x72, 0x76, 0x56, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x22, 0x2d, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, - 0x6c, 0x6c, 0x73, 0x22, 0xc5, 0x01, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x56, - 0x0a, 0x0d, 0x73, 0x72, 0x76, 0x5f, 0x76, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x73, 0x72, 0x76, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x1a, 0x53, 0x0a, 0x10, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 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, 0x29, 0x0a, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x76, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4c, 0x0a, 0x10, 0x47, - 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x3d, 0x0a, 0x11, 0x47, 0x65, 0x74, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x28, - 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, - 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0xe8, 0x01, 0x0a, 0x11, 0x47, 0x65, 0x74, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x12, 0x3c, - 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, - 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0d, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x35, 0x0a, 0x0b, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, - 0x79, 0x70, 0x65, 0x22, 0x40, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x07, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x07, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0x2c, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, - 0x6c, 0x6f, 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, - 0x61, 0x74, 0x68, 0x22, 0x46, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, - 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, - 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, - 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x66, 0x0a, 0x0c, 0x54, - 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, - 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, - 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, - 0x72, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, - 0x72, 0x65, 0x6e, 0x22, 0x2f, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x22, 0x4d, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, + 0x23, 0x0a, 0x0d, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, + 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, + 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, + 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, + 0x43, 0x0a, 0x19, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, + 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x73, 0x22, 0x5b, 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x22, 0x16, 0x0a, 0x14, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7f, 0x0a, 0x19, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x09, 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x52, 0x65, + 0x6d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x16, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, + 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, + 0x6c, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x63, 0x75, + 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, 0x65, 0x63, + 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x22, 0x19, 0x0a, 0x17, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x46, 0x0a, 0x15, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0x7b, 0x0a, 0x16, 0x52, 0x65, 0x70, + 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x2f, 0x0a, 0x07, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x07, 0x70, + 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x82, 0x02, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x74, 0x6f, + 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2d, 0x0a, + 0x0b, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x52, 0x0a, 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, + 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x6f, 0x50, + 0x6f, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x12, 0x3e, 0x0a, 0x14, 0x72, + 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, + 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x12, 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, + 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0xad, 0x01, 0x0a, 0x19, + 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, - 0x69, 0x61, 0x73, 0x22, 0x2e, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x22, 0x42, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x5f, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x07, - 0x76, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x8b, 0x01, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x69, 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x24, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x4d, 0x0a, 0x1b, 0x52, + 0x65, 0x74, 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0xdd, 0x01, 0x0a, 0x1c, 0x52, + 0x65, 0x74, 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x75, 0x0a, 0x16, 0x72, + 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, + 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, + 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 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, 0x04, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x51, 0x0a, 0x15, 0x52, 0x75, + 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x18, 0x0a, + 0x16, 0x52, 0x75, 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6d, 0x0a, 0x22, 0x53, 0x65, 0x74, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x22, 0x55, 0x0a, 0x23, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, + 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xc8, 0x01, + 0x0a, 0x1c, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, + 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, + 0x27, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x4f, 0x0a, 0x1d, 0x53, 0x65, 0x74, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, + 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x5e, 0x0a, 0x1e, 0x53, 0x65, 0x74, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x4a, 0x04, 0x08, + 0x02, 0x10, 0x03, 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x22, 0x51, 0x0a, 0x1f, 0x53, 0x65, 0x74, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x72, 0x0a, 0x1f, + 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x61, - 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x1b, 0x0a, 0x09, - 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x08, 0x6e, 0x61, 0x6d, 0x65, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x22, 0x49, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, - 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x13, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, - 0x22, 0xfb, 0x01, 0x0a, 0x17, 0x49, 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, - 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, + 0x22, 0x49, 0x0a, 0x20, 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x73, 0x50, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x8e, 0x02, 0x0a, 0x1c, + 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x52, - 0x0a, 0x1a, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x17, 0x70, 0x72, 0x69, 0x6d, 0x61, - 0x72, 0x79, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, - 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, - 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, - 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0x42, - 0x0a, 0x18, 0x49, 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, - 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, - 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x22, 0x4e, 0x0a, 0x1c, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, - 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, - 0x69, 0x64, 0x22, 0xdf, 0x01, 0x0a, 0x1d, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x16, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x61, 0x66, 0x66, - 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x4c, 0x61, 0x75, 0x6e, 0x63, 0x68, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, - 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, - 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x46, 0x0a, 0x18, - 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, - 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9a, 0x06, 0x0a, 0x17, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x27, 0x0a, 0x0f, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, - 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, - 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, - 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x6c, 0x0a, - 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, - 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, - 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x08, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, - 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, - 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, - 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x32, 0x0a, - 0x15, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, - 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x65, 0x78, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x28, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x5a, 0x6f, 0x6e, 0x65, 0x12, 0x15, 0x0a, 0x06, 0x6f, - 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x6e, 0x44, - 0x64, 0x6c, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, - 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x74, 0x6f, - 0x70, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x2a, 0x0a, 0x11, 0x64, 0x72, - 0x6f, 0x70, 0x5f, 0x66, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, - 0x0f, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x46, 0x6f, 0x72, 0x65, 0x69, - 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x64, 0x65, 0x66, 0x65, 0x72, 0x5f, - 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x10, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, 0x66, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, - 0x64, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, 0x6f, - 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, - 0x74, 0x6f, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x28, 0x0a, 0x10, 0x6e, 0x6f, 0x5f, 0x72, 0x6f, - 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0e, 0x6e, 0x6f, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, - 0x73, 0x22, 0xd5, 0x01, 0x0a, 0x18, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, - 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x48, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, - 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, - 0x6c, 0x73, 0x1a, 0x55, 0x0a, 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, - 0x18, 0x0a, 0x07, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x07, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x22, 0xe9, 0x01, 0x0a, 0x19, 0x4d, 0x6f, - 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, - 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1b, 0x0a, 0x09, - 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x08, 0x6b, 0x65, 0x65, 0x70, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x12, 0x6b, 0x65, 0x65, - 0x70, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, 0x65, 0x70, 0x52, 0x6f, 0x75, 0x74, 0x69, - 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6e, 0x61, 0x6d, - 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, - 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x17, 0x0a, 0x07, - 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, - 0x72, 0x79, 0x52, 0x75, 0x6e, 0x22, 0x5e, 0x0a, 0x1a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, - 0x0f, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x4d, 0x0a, 0x11, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x35, + 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x64, + 0x65, 0x6e, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0c, 0x64, 0x65, 0x6e, 0x69, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x12, 0x32, 0x0a, 0x15, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x13, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x22, 0x46, 0x0a, 0x1d, + 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, + 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, + 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x22, 0x6a, 0x0a, 0x12, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x22, 0x14, 0x0a, 0x12, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x89, 0x02, 0x0a, 0x1b, 0x50, - 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, - 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x12, 0x3a, 0x0a, 0x0d, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x5f, 0x70, 0x72, - 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, - 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x52, 0x0c, 0x61, 0x76, 0x6f, 0x69, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x12, 0x44, 0x0a, 0x15, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x13, 0x77, 0x61, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x54, - 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0xba, 0x01, 0x0a, 0x1c, 0x50, 0x6c, 0x61, 0x6e, 0x6e, - 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x40, 0x0a, 0x10, 0x70, 0x72, 0x6f, - 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0f, 0x70, 0x72, 0x6f, 0x6d, - 0x6f, 0x74, 0x65, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x26, 0x0a, 0x06, 0x65, - 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, - 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, - 0x6e, 0x74, 0x73, 0x22, 0x74, 0x0a, 0x1b, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, - 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, - 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, - 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x22, 0x1e, 0x0a, 0x1c, 0x52, 0x65, 0x62, - 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, - 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x1a, 0x52, 0x65, 0x62, - 0x75, 0x69, 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x1d, 0x0a, - 0x1b, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, - 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4f, 0x0a, 0x13, - 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, - 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x16, 0x0a, - 0x14, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x64, 0x0a, 0x1a, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, - 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x83, 0x01, 0x0a, 0x1b, - 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x69, - 0x73, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, - 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x69, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x61, 0x6c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x17, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, 0x64, 0x65, 0x74, - 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x61, 0x6c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, - 0x73, 0x22, 0x4f, 0x0a, 0x13, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, + 0x6c, 0x69, 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x22, 0x15, 0x0a, 0x13, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x88, 0x01, 0x0a, 0x1a, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa9, 0x01, 0x0a, 0x1b, 0x52, - 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, - 0x61, 0x69, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x69, - 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x50, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, - 0x6e, 0x63, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, - 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, 0x46, 0x0a, 0x1c, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, - 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xbc, - 0x01, 0x0a, 0x18, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x0a, - 0x0d, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x61, 0x69, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x70, 0x72, - 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, - 0x6c, 0x75, 0x64, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0b, 0x63, - 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, 0x43, 0x0a, - 0x19, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, - 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x22, 0x5b, 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, - 0x16, 0x0a, 0x14, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7f, 0x0a, 0x19, 0x52, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x63, 0x65, 0x6c, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, - 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, - 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65, 0x22, 0x1c, 0x0a, 0x1a, 0x52, 0x65, 0x6d, 0x6f, - 0x76, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x16, 0x52, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x0a, - 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x73, 0x68, 0x61, 0x72, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, - 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x65, 0x6c, 0x6c, - 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x63, 0x75, 0x72, 0x73, - 0x69, 0x76, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x72, 0x65, 0x63, 0x75, 0x72, - 0x73, 0x69, 0x76, 0x65, 0x22, 0x19, 0x0a, 0x17, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x46, 0x0a, 0x15, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x73, 0x22, 0x1d, 0x0a, 0x1b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x62, 0x0a, 0x1a, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x22, 0x54, 0x0a, 0x1b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x35, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x54, 0x0a, 0x20, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x22, 0xaa, 0x03, 0x0a, 0x21, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x78, 0x0a, 0x14, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, + 0x73, 0x12, 0x5a, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x6d, 0x61, 0x70, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x1a, 0x5f, 0x0a, + 0x18, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2d, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x72, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x4e, + 0x0a, 0x0e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x26, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8b, + 0x01, 0x0a, 0x1d, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, + 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0x7b, 0x0a, 0x16, 0x52, 0x65, 0x70, 0x61, 0x72, - 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x12, 0x2f, 0x0a, 0x07, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x07, 0x70, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x22, 0x82, 0x02, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, - 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, - 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2d, 0x0a, 0x0b, 0x62, - 0x61, 0x63, 0x6b, 0x75, 0x70, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0a, - 0x62, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x72, 0x65, - 0x73, 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x6f, 0x50, 0x6f, 0x73, - 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x12, 0x3e, 0x0a, 0x14, 0x72, 0x65, 0x73, - 0x74, 0x6f, 0x72, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, - 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x12, 0x72, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x6f, - 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0xad, 0x01, 0x0a, 0x19, 0x52, 0x65, - 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x12, 0x24, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x6c, 0x6f, 0x67, 0x75, 0x74, 0x69, 0x6c, 0x2e, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x4d, 0x0a, 0x1b, 0x52, 0x65, 0x74, - 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0xdd, 0x01, 0x0a, 0x1c, 0x52, 0x65, 0x74, - 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x75, 0x0a, 0x16, 0x72, 0x6f, 0x77, - 0x73, 0x5f, 0x61, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x2e, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, - 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x6f, 0x77, - 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x1a, 0x46, 0x0a, 0x18, 0x52, 0x6f, 0x77, 0x73, 0x41, 0x66, 0x66, 0x65, 0x63, 0x74, 0x65, 0x64, - 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 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, 0x04, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x51, 0x0a, 0x15, 0x52, 0x75, 0x6e, 0x48, - 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, - 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x52, - 0x75, 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6d, 0x0a, 0x22, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, - 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x2b, 0x0a, 0x11, 0x64, 0x75, 0x72, 0x61, 0x62, - 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x70, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x10, 0x64, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, - 0x6c, 0x69, 0x63, 0x79, 0x22, 0x55, 0x0a, 0x23, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x50, 0x6f, 0x6c, - 0x69, 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x1c, - 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, - 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, - 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, - 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x27, 0x0a, - 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x4f, 0x0a, 0x1d, 0x53, 0x65, 0x74, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x65, 0x72, 0x76, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x5e, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x4a, 0x04, 0x08, 0x02, 0x10, - 0x03, 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x22, 0x51, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2e, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x72, 0x0a, 0x1f, 0x53, 0x65, - 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x20, 0x0a, 0x1e, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7c, + 0x0a, 0x12, 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, + 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, + 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, + 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2c, + 0x0a, 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, + 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0xf0, 0x01, 0x0a, 0x15, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x41, 0x64, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, - 0x1d, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x22, 0x49, - 0x0a, 0x20, 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, 0x73, 0x50, 0x72, 0x69, 0x6d, - 0x61, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x8e, 0x02, 0x0a, 0x1c, 0x53, 0x65, - 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x35, 0x0a, 0x0b, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x65, 0x6e, - 0x69, 0x65, 0x64, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x0c, 0x64, 0x65, 0x6e, 0x69, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x32, - 0x0a, 0x15, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, - 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x64, - 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x22, 0x46, 0x0a, 0x1d, 0x53, 0x65, - 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x22, 0x6a, 0x0a, 0x12, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, - 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x77, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x15, - 0x0a, 0x13, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x88, 0x01, 0x0a, 0x1a, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, + 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x75, 0x69, + 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x2f, 0x0a, + 0x09, 0x6b, 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, + 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0x3f, 0x0a, 0x16, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x5e, 0x0a, 0x18, 0x53, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x03, 0x75, 0x69, 0x64, 0x22, 0x42, 0x0a, 0x19, 0x53, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x53, 0x0a, 0x17, 0x53, + 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, + 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x22, 0x1a, 0x0a, 0x18, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x16, + 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x22, 0x1d, 0x0a, 0x1b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x62, 0x0a, 0x1a, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x22, 0x19, 0x0a, 0x17, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x21, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x52, + 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, + 0xc6, 0x01, 0x0a, 0x22, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x6c, 0x79, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, + 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, + 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, + 0x12, 0x36, 0x0a, 0x0b, 0x6f, 0x6c, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6f, 0x6c, + 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x5c, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, + 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x5d, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, + 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x64, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, + 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 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, 0x12, 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x5f, 0x61, 0x6c, + 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, + 0x0a, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x65, 0x0a, 0x18, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x63, + 0x65, 0x6c, 0x6c, 0x73, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, + 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x22, 0x34, 0x0a, 0x0f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, + 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0xfb, 0x01, 0x0a, 0x10, 0x56, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x62, 0x0a, 0x13, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x1a, 0x69, 0x0a, 0x16, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x58, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, + 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, + 0x22, 0xfc, 0x01, 0x0a, 0x18, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x61, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x37, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, + 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0xd8, 0x01, 0x0a, 0x1d, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, + 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, + 0x76, 0x69, 0x65, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, + 0x6c, 0x75, 0x64, 0x65, 0x56, 0x69, 0x65, 0x77, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x6b, 0x69, + 0x70, 0x5f, 0x6e, 0x6f, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0d, 0x73, 0x6b, 0x69, 0x70, 0x4e, 0x6f, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x56, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x88, 0x02, 0x0a, 0x1e, 0x56, + 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x67, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x3d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6b, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, - 0x12, 0x0a, 0x04, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, - 0x65, 0x6c, 0x6c, 0x22, 0x54, 0x0a, 0x1b, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x35, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x54, 0x0a, 0x20, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, - 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, - 0xaa, 0x03, 0x0a, 0x21, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x78, 0x0a, 0x14, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x72, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x12, - 0x5a, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x1a, 0x5f, 0x0a, 0x18, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2d, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x4e, 0x0a, 0x0e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, - 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, - 0x12, 0x26, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x10, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8b, 0x01, 0x0a, - 0x1d, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, - 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0b, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x20, 0x0a, 0x1e, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7c, 0x0a, 0x12, - 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x69, - 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, - 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2c, 0x0a, 0x08, - 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, - 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x08, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x53, 0x6c, - 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0xf0, 0x01, 0x0a, 0x15, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x41, 0x64, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x10, 0x0a, - 0x03, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, - 0x27, 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x2f, 0x0a, 0x09, 0x6b, - 0x65, 0x79, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x73, 0x22, 0x3f, 0x0a, 0x16, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, - 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, - 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x5e, 0x0a, 0x18, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x03, 0x75, 0x69, 0x64, 0x22, 0x42, 0x0a, 0x19, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x53, 0x0a, 0x17, 0x53, 0x74, 0x61, - 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, - 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x1a, - 0x0a, 0x18, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x16, 0x53, 0x74, - 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x61, - 0x6c, 0x69, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, - 0x73, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x19, - 0x0a, 0x17, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x52, 0x0a, 0x21, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x52, 0x65, 0x70, - 0x61, 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, - 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, - 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x22, 0xc6, 0x01, - 0x0a, 0x22, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x6c, 0x79, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x6e, 0x65, 0x77, 0x5f, 0x70, 0x72, - 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, - 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x36, - 0x0a, 0x0b, 0x6f, 0x6c, 0x64, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x6f, 0x6c, 0x64, 0x50, - 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x5c, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, - 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x5d, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, - 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, - 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x09, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x63, 0x65, 0x6c, 0x6c, 0x49, - 0x6e, 0x66, 0x6f, 0x22, 0x64, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, - 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 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, 0x12, 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x5f, 0x61, 0x6c, 0x69, 0x61, - 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x63, - 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x22, 0x65, 0x0a, 0x18, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x0b, 0x63, 0x65, 0x6c, - 0x6c, 0x73, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, - 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, - 0x6c, 0x69, 0x61, 0x73, 0x52, 0x0a, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x22, 0x34, 0x0a, 0x0f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0xfb, 0x01, 0x0a, 0x10, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x62, 0x0a, 0x13, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x5f, 0x62, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, - 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x1a, 0x69, 0x0a, 0x16, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x58, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, - 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, - 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x22, 0xfc, - 0x01, 0x0a, 0x18, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, + 0x21, 0x0a, 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x73, 0x22, 0x31, 0x0a, 0x15, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x61, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x37, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, - 0x64, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, - 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd8, 0x01, - 0x0a, 0x1d, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x3c, 0x0a, 0x1e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x22, 0x8a, 0x02, 0x0a, 0x1f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x12, 0x68, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, + 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x4f, 0x0a, 0x1b, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x65, - 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x69, - 0x65, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, - 0x64, 0x65, 0x56, 0x69, 0x65, 0x77, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x6b, 0x69, 0x70, 0x5f, - 0x6e, 0x6f, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0d, 0x73, 0x6b, 0x69, 0x70, 0x4e, 0x6f, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, - 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, - 0x65, 0x56, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x88, 0x02, 0x0a, 0x1e, 0x56, 0x61, 0x6c, - 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x67, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x3d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, - 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x22, 0x38, 0x0a, 0x1c, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x98, 0x01, 0x0a, 0x16, + 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x69, 0x65, + 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x56, 0x69, 0x65, 0x77, 0x73, 0x22, 0xfa, 0x01, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x60, 0x0a, 0x10, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, @@ -15158,234 +15244,169 @@ var file_vtctldata_proto_rawDesc = []byte{ 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x22, 0x6b, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x21, 0x0a, - 0x0c, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0b, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, - 0x22, 0x31, 0x0a, 0x15, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x22, 0x3c, 0x0a, 0x1e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x22, 0x8a, 0x02, 0x0a, 0x1f, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, - 0x68, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4f, - 0x0a, 0x1b, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x02, 0x38, 0x01, 0x22, 0x9a, 0x01, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, - 0x38, 0x0a, 0x1c, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x98, 0x01, 0x0a, 0x16, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, - 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x73, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x56, - 0x69, 0x65, 0x77, 0x73, 0x22, 0xfa, 0x01, 0x0a, 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x60, 0x0a, 0x10, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, - 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x1a, 0x63, 0x0a, 0x13, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x9a, 0x01, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x12, 0x1b, 0x0a, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x64, 0x61, 0x74, 0x61, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6b, 0x65, 0x65, 0x70, 0x44, 0x61, 0x74, 0x61, - 0x12, 0x2c, 0x0a, 0x12, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, - 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x6b, 0x65, - 0x65, 0x70, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0xd1, - 0x01, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, - 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, - 0x61, 0x72, 0x79, 0x12, 0x46, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x55, 0x0a, 0x0a, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x64, 0x22, 0x4f, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, - 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x22, 0xc1, 0x07, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, - 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, 0x74, 0x61, - 0x74, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x1b, 0x0a, 0x09, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6b, 0x65, 0x65, 0x70, 0x44, 0x61, + 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x12, 0x6b, 0x65, 0x65, 0x70, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, + 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, + 0x6b, 0x65, 0x65, 0x70, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, + 0x22, 0xd1, 0x01, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, + 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, + 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x46, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x55, 0x0a, + 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x22, 0x4f, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x22, 0xc1, 0x07, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x5f, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x58, 0x0a, 0x0d, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, - 0x58, 0x0a, 0x0d, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x1a, 0xe8, 0x01, 0x0a, 0x0e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, + 0x1d, 0x0a, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x72, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x27, + 0x0a, 0x0f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0e, 0x72, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, + 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x79, 0x74, 0x65, 0x73, + 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x62, + 0x79, 0x74, 0x65, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x79, + 0x74, 0x65, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0a, 0x62, 0x79, 0x74, 0x65, 0x73, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x29, 0x0a, 0x10, 0x62, + 0x79, 0x74, 0x65, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x50, 0x65, 0x72, 0x63, + 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x1a, 0xbc, 0x01, 0x0a, 0x10, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, + 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x12, 0x12, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x1a, 0x5c, 0x0a, 0x0c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x73, 0x12, 0x4c, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x1a, 0xe8, 0x01, 0x0a, 0x0e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, - 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x1d, 0x0a, - 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x09, 0x72, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x27, 0x0a, 0x0f, - 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x0e, 0x72, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x63, 0x65, - 0x6e, 0x74, 0x61, 0x67, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, - 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x62, 0x79, 0x74, - 0x65, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x79, 0x74, 0x65, - 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x62, - 0x79, 0x74, 0x65, 0x73, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x12, 0x29, 0x0a, 0x10, 0x62, 0x79, 0x74, - 0x65, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x61, 0x67, 0x65, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x02, 0x52, 0x0f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, - 0x74, 0x61, 0x67, 0x65, 0x1a, 0xbc, 0x01, 0x0a, 0x10, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x70, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, - 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, - 0x12, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x69, - 0x6e, 0x66, 0x6f, 0x1a, 0x5c, 0x0a, 0x0c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x73, 0x12, 0x4c, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x73, 0x1a, 0x73, 0x0a, 0x13, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x46, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, - 0x74, 0x72, 0x65, 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, 0x44, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd7, 0x03, 0x0a, 0x1c, 0x57, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x77, 0x69, 0x74, 0x63, 0x68, 0x54, 0x72, 0x61, 0x66, 0x66, 0x69, - 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, - 0x4f, 0x0a, 0x1b, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x5f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, - 0x12, 0x3c, 0x0a, 0x1a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x76, 0x65, 0x72, - 0x73, 0x65, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x76, 0x65, - 0x72, 0x73, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1c, - 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x07, - 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, - 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, 0x79, 0x5f, - 0x72, 0x75, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, 0x52, 0x75, - 0x6e, 0x12, 0x3e, 0x0a, 0x1b, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x5f, - 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, - 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x19, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x69, - 0x7a, 0x65, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, - 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x1d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x77, - 0x69, 0x74, 0x63, 0x68, 0x54, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x1f, 0x0a, - 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x23, - 0x0a, 0x0d, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x5f, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x64, 0x72, - 0x79, 0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x90, 0x01, 0x0a, 0x15, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x5b, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, - 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xd1, - 0x01, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, - 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, - 0x61, 0x72, 0x79, 0x12, 0x46, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x55, 0x0a, 0x0a, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x06, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, 0x70, 0x6f, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, - 0x67, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x67, - 0x65, 0x64, 0x2a, 0x4a, 0x0a, 0x15, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x12, 0x0a, 0x0a, 0x06, 0x43, - 0x55, 0x53, 0x54, 0x4f, 0x4d, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x4f, 0x56, 0x45, 0x54, - 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x52, 0x45, 0x41, 0x54, - 0x45, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x02, 0x2a, 0x38, - 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, - 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x41, 0x53, 0x43, - 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x44, 0x45, 0x53, 0x43, - 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x42, 0x28, 0x5a, 0x26, 0x76, 0x69, 0x74, 0x65, - 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, - 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x73, 0x1a, 0x73, 0x0a, 0x13, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x46, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x70, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x53, 0x74, 0x72, 0x65, 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, + 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xd7, 0x03, 0x0a, 0x1c, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x77, 0x69, 0x74, 0x63, 0x68, 0x54, 0x72, 0x61, 0x66, + 0x66, 0x69, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, + 0x73, 0x12, 0x4f, 0x0a, 0x1b, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x5f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, + 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 0x41, 0x6c, 0x6c, 0x6f, 0x77, + 0x65, 0x64, 0x12, 0x3c, 0x0a, 0x1a, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x76, + 0x65, 0x72, 0x73, 0x65, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x65, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, + 0x76, 0x65, 0x72, 0x73, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, + 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x72, + 0x79, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x64, 0x72, 0x79, + 0x52, 0x75, 0x6e, 0x12, 0x3e, 0x0a, 0x1b, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x69, 0x7a, + 0x65, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, + 0x65, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x19, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, + 0x6c, 0x69, 0x7a, 0x65, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, + 0x63, 0x65, 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x1d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x53, 0x77, 0x69, 0x74, 0x63, 0x68, 0x54, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, + 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x72, 0x79, 0x5f, 0x72, 0x75, 0x6e, + 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, + 0x64, 0x72, 0x79, 0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x90, 0x01, + 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x12, 0x5b, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x52, 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x22, 0xd1, 0x01, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, + 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, + 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x46, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x1a, 0x55, 0x0a, + 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x06, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x74, 0x6f, + 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, + 0x61, 0x6e, 0x67, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x63, 0x68, 0x61, + 0x6e, 0x67, 0x65, 0x64, 0x2a, 0x4a, 0x0a, 0x15, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, + 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x12, 0x0a, 0x0a, + 0x06, 0x43, 0x55, 0x53, 0x54, 0x4f, 0x4d, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x4f, 0x56, + 0x45, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x52, 0x45, + 0x41, 0x54, 0x45, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x02, + 0x2a, 0x38, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x41, + 0x53, 0x43, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x44, 0x45, + 0x53, 0x43, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x42, 0x28, 0x5a, 0x26, 0x76, 0x69, + 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, + 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go index 973a3610d05..effc0ddf947 100644 --- a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go +++ b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go @@ -4823,6 +4823,18 @@ func (m *MaterializeSettings) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.AtomicCopy { + i-- + if m.AtomicCopy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x80 + } if m.TabletSelectionPreference != 0 { i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) i-- @@ -11407,6 +11419,18 @@ func (m *MoveTablesCreateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, erro i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.AtomicCopy { + i-- + if m.AtomicCopy { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x98 + } if m.NoRoutingRules { i-- if m.NoRoutingRules { @@ -16943,6 +16967,9 @@ func (m *MaterializeSettings) SizeVT() (n int) { if m.TabletSelectionPreference != 0 { n += 1 + sov(uint64(m.TabletSelectionPreference)) } + if m.AtomicCopy { + n += 3 + } n += len(m.unknownFields) return n } @@ -19413,6 +19440,9 @@ func (m *MoveTablesCreateRequest) SizeVT() (n int) { if m.NoRoutingRules { n += 3 } + if m.AtomicCopy { + n += 3 + } n += len(m.unknownFields) return n } @@ -22172,6 +22202,26 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { break } } + case 16: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AtomicCopy", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AtomicCopy = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -38358,6 +38408,26 @@ func (m *MoveTablesCreateRequest) UnmarshalVT(dAtA []byte) error { } } m.NoRoutingRules = bool(v != 0) + case 19: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AtomicCopy", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AtomicCopy = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index 4c5cff2023c..77b7f267a30 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -707,6 +707,16 @@ func (itc *internalTabletConn) VStreamRows( return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) } +// VStreamTables is part of the QueryService interface. +func (itc *internalTabletConn) VStreamTables( + ctx context.Context, + request *binlogdatapb.VStreamTablesRequest, + send func(*binlogdatapb.VStreamTablesResponse) error, +) error { + err := itc.tablet.qsc.QueryService().VStreamTables(ctx, request, send) + return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) +} + // VStreamResults is part of the QueryService interface. func (itc *internalTabletConn) VStreamResults( ctx context.Context, diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index 402bfeb14e8..e9583e7c5b0 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -2099,6 +2099,7 @@ func commandVReplicationWorkflow(ctx context.Context, wr *wrangler.Wrangler, sub stopAfterCopy := subFlags.Bool("stop_after_copy", false, "Streams will be stopped once the copy phase is completed") dropForeignKeys := subFlags.Bool("drop_foreign_keys", false, "If true, tables in the target keyspace will be created without foreign keys.") maxReplicationLagAllowed := subFlags.Duration("max_replication_lag_allowed", defaultMaxReplicationLagAllowed, "Allow traffic to be switched only if vreplication lag is below this (in seconds)") + atomicCopy := subFlags.Bool("atomic-copy", false, "(EXPERIMENTAL) Use this if your source keyspace has tables which use foreign key constraints. All tables from the source will be moved.") onDDL := "IGNORE" subFlags.StringVar(&onDDL, "on-ddl", onDDL, "What to do when DDL is encountered in the VReplication stream. Possible values are IGNORE, STOP, EXEC, and EXEC_IGNORE.") @@ -2159,6 +2160,7 @@ func commandVReplicationWorkflow(ctx context.Context, wr *wrangler.Wrangler, sub DryRun: *dryRun, AutoStart: *autoStart, StopAfterCopy: *stopAfterCopy, + AtomicCopy: *atomicCopy, } printDetails := func() error { @@ -2250,6 +2252,24 @@ func commandVReplicationWorkflow(ctx context.Context, wr *wrangler.Wrangler, sub wr.Logger().Errorf("keyspace %s not found", *sourceKeyspace) return err } + + if *atomicCopy { + var errors []string + if !*allTables { + errors = append(errors, "atomic copy requires --all.") + } + if *tables != "" { + errors = append(errors, "atomic copy does not support specifying tables.") + } + if *excludes != "" { + errors = append(errors, "atomic copy does not support specifying excludes.") + } + if len(errors) > 0 { + errors = append(errors, "Found options incompatible with atomic copy:") + return fmt.Errorf(strings.Join(errors, " ")) + } + } + if !*allTables && *tables == "" { return fmt.Errorf("no tables specified to move") } diff --git a/go/vt/vtctl/workflow/materializer.go b/go/vt/vtctl/workflow/materializer.go index 54a6a726bcf..2e48da05ed8 100644 --- a/go/vt/vtctl/workflow/materializer.go +++ b/go/vt/vtctl/workflow/materializer.go @@ -61,6 +61,20 @@ type materializer struct { isPartial bool } +func (mz *materializer) getWorkflowSubType() (binlogdatapb.VReplicationWorkflowSubType, error) { + switch { + case mz.isPartial && mz.ms.AtomicCopy: + return binlogdatapb.VReplicationWorkflowSubType_None, + fmt.Errorf("both atomic copy and partial mode cannot be specified for the same workflow") + case mz.isPartial: + return binlogdatapb.VReplicationWorkflowSubType_Partial, nil + case mz.ms.AtomicCopy: + return binlogdatapb.VReplicationWorkflowSubType_AtomicCopy, nil + default: + return binlogdatapb.VReplicationWorkflowSubType_None, nil + } +} + func (mz *materializer) prepareMaterializerStreams(req *vtctldatapb.MoveTablesCreateRequest) error { if err := validateNewWorkflow(mz.ctx, mz.ts, mz.tmc, mz.ms.TargetKeyspace, mz.ms.Workflow); err != nil { return err @@ -72,10 +86,13 @@ func (mz *materializer) prepareMaterializerStreams(req *vtctldatapb.MoveTablesCr if err := mz.deploySchema(); err != nil { return err } - workflowSubType := binlogdatapb.VReplicationWorkflowSubType_None - if mz.isPartial { - workflowSubType = binlogdatapb.VReplicationWorkflowSubType_Partial + + var workflowSubType binlogdatapb.VReplicationWorkflowSubType + workflowSubType, err = mz.getWorkflowSubType() + if err != nil { + return err } + return mz.forAllTargets(func(target *topo.ShardInfo) error { targetPrimary, err := mz.ts.GetTablet(mz.ctx, target.PrimaryAlias) if err != nil { diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index 9d79156ae46..9d6b2a1258e 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -1040,6 +1040,7 @@ func (s *Server) MoveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl SourceShards: req.SourceShards, OnDdl: req.OnDdl, DeferSecondaryKeys: req.DeferSecondaryKeys, + AtomicCopy: req.AtomicCopy, } if req.SourceTimeZone != "" { ms.SourceTimeZone = req.SourceTimeZone diff --git a/go/vt/vtgate/endtoend/vstream_test.go b/go/vt/vtgate/endtoend/vstream_test.go index c96f84faff1..42dd6e3d2a3 100644 --- a/go/vt/vtgate/endtoend/vstream_test.go +++ b/go/vt/vtgate/endtoend/vstream_test.go @@ -155,6 +155,7 @@ func TestVStream(t *testing.T) { Values: []byte("11"), }, }}, + Flags: 1, // foreign_key_checks are enabled by default. } gotRows := events[2].RowEvent if !proto.Equal(gotRows, wantRows) { @@ -477,6 +478,7 @@ func TestVStreamCopyResume(t *testing.T) { case nil: for _, ev := range e { if ev.Type == binlogdatapb.VEventType_ROW { + ev.RowEvent.Flags = 0 // null Flags, so we don't have to define flags in every wanted row event. evs = append(evs, ev) if ev.Timestamp == 0 { rowCopyEvents++ diff --git a/go/vt/vttablet/endtoend/vstreamer_test.go b/go/vt/vttablet/endtoend/vstreamer_test.go index 75095588d39..312273e0c84 100644 --- a/go/vt/vttablet/endtoend/vstreamer_test.go +++ b/go/vt/vttablet/endtoend/vstreamer_test.go @@ -384,6 +384,10 @@ func expectLogs(ctx context.Context, t *testing.T, query string, eventCh chan [] if ev.Type == binlogdatapb.VEventType_HEARTBEAT { continue } + if ev.Type == binlogdatapb.VEventType_ROW { + ev.RowEvent.Flags = 0 // null Flags, so we don't have to define flags in every wanted row event. + } + if ev.Throttled { continue } diff --git a/go/vt/vttablet/grpcqueryservice/server.go b/go/vt/vttablet/grpcqueryservice/server.go index 49557d36ef3..64a7697162d 100644 --- a/go/vt/vttablet/grpcqueryservice/server.go +++ b/go/vt/vttablet/grpcqueryservice/server.go @@ -352,6 +352,16 @@ func (q *query) VStreamRows(request *binlogdatapb.VStreamRowsRequest, stream que return vterrors.ToGRPC(err) } +func (q *query) VStreamTables(request *binlogdatapb.VStreamTablesRequest, stream queryservicepb.Query_VStreamTablesServer) (err error) { + defer q.server.HandlePanic(&err) + ctx := callerid.NewContext(callinfo.GRPCCallInfo(stream.Context()), + request.EffectiveCallerId, + request.ImmediateCallerId, + ) + err = q.server.VStreamTables(ctx, request, stream.Send) + return vterrors.ToGRPC(err) +} + // VStreamResults is part of the queryservice.QueryServer interface func (q *query) VStreamResults(request *binlogdatapb.VStreamResultsRequest, stream queryservicepb.Query_VStreamResultsServer) (err error) { defer q.server.HandlePanic(&err) diff --git a/go/vt/vttablet/grpctabletconn/conn.go b/go/vt/vttablet/grpctabletconn/conn.go index 7ef533f580d..cb97abcbbae 100644 --- a/go/vt/vttablet/grpctabletconn/conn.go +++ b/go/vt/vttablet/grpctabletconn/conn.go @@ -735,6 +735,46 @@ func (conn *gRPCQueryClient) VStreamRows(ctx context.Context, request *binlogdat } } +// VStreamTables streams rows of a query from the specified starting point. +func (conn *gRPCQueryClient) VStreamTables(ctx context.Context, request *binlogdatapb.VStreamTablesRequest, send func(*binlogdatapb.VStreamTablesResponse) error) error { + stream, err := func() (queryservicepb.Query_VStreamTablesClient, error) { + conn.mu.RLock() + defer conn.mu.RUnlock() + if conn.cc == nil { + return nil, tabletconn.ConnClosed + } + + req := &binlogdatapb.VStreamTablesRequest{ + Target: request.Target, + EffectiveCallerId: callerid.EffectiveCallerIDFromContext(ctx), + ImmediateCallerId: callerid.ImmediateCallerIDFromContext(ctx), + } + stream, err := conn.c.VStreamTables(ctx, req) + if err != nil { + return nil, tabletconn.ErrorFromGRPC(err) + } + return stream, nil + }() + if err != nil { + return err + } + r := binlogdatapb.VStreamTablesResponseFromVTPool() + defer r.ReturnToVTPool() + for { + err := stream.RecvMsg(r) + if err != nil { + return tabletconn.ErrorFromGRPC(err) + } + if ctx.Err() != nil { + return ctx.Err() + } + if err := send(r); err != nil { + return err + } + r.ResetVT() + } +} + // VStreamResults streams rows of a query from the specified starting point. func (conn *gRPCQueryClient) VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { stream, err := func() (queryservicepb.Query_VStreamResultsClient, error) { diff --git a/go/vt/vttablet/queryservice/queryservice.go b/go/vt/vttablet/queryservice/queryservice.go index 1b9bdac13e9..c4d72b0c927 100644 --- a/go/vt/vttablet/queryservice/queryservice.go +++ b/go/vt/vttablet/queryservice/queryservice.go @@ -98,6 +98,10 @@ type QueryService interface { // VStreamRows streams rows of a table from the specified starting point. VStreamRows(ctx context.Context, request *binlogdatapb.VStreamRowsRequest, send func(*binlogdatapb.VStreamRowsResponse) error) error + // VStreamTables streams rows of all tables + + VStreamTables(ctx context.Context, request *binlogdatapb.VStreamTablesRequest, send func(*binlogdatapb.VStreamTablesResponse) error) error + // VStreamResults streams results along with the gtid of the snapshot. VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error diff --git a/go/vt/vttablet/queryservice/wrapped.go b/go/vt/vttablet/queryservice/wrapped.go index 376c228b02a..910a1d8948e 100644 --- a/go/vt/vttablet/queryservice/wrapped.go +++ b/go/vt/vttablet/queryservice/wrapped.go @@ -254,6 +254,13 @@ func (ws *wrappedService) VStreamRows(ctx context.Context, request *binlogdatapb }) } +func (ws *wrappedService) VStreamTables(ctx context.Context, request *binlogdatapb.VStreamTablesRequest, send func(response *binlogdatapb.VStreamTablesResponse) error) error { + return ws.wrapper(ctx, request.Target, ws.impl, "VStreamTables", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) { + innerErr := conn.VStreamTables(ctx, request, send) + return false, innerErr + }) +} + func (ws *wrappedService) VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { return ws.wrapper(ctx, target, ws.impl, "VStreamResults", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) { innerErr := conn.VStreamResults(ctx, target, query, send) diff --git a/go/vt/vttablet/sandboxconn/sandboxconn.go b/go/vt/vttablet/sandboxconn/sandboxconn.go index a654d18cea5..b58a793db43 100644 --- a/go/vt/vttablet/sandboxconn/sandboxconn.go +++ b/go/vt/vttablet/sandboxconn/sandboxconn.go @@ -503,6 +503,11 @@ func (sbc *SandboxConn) VStreamRows(ctx context.Context, request *binlogdatapb.V return fmt.Errorf("not implemented in test") } +// VStreamTables is part of the QueryService interface. +func (sbc *SandboxConn) VStreamTables(ctx context.Context, request *binlogdatapb.VStreamTablesRequest, send func(response *binlogdatapb.VStreamTablesResponse) error) error { + return fmt.Errorf("not implemented in test") +} + // VStreamResults is part of the QueryService interface. func (sbc *SandboxConn) VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { return fmt.Errorf("not implemented in test") diff --git a/go/vt/vttablet/tabletconntest/fakequeryservice.go b/go/vt/vttablet/tabletconntest/fakequeryservice.go index 0e089a1882e..cfe540ead42 100644 --- a/go/vt/vttablet/tabletconntest/fakequeryservice.go +++ b/go/vt/vttablet/tabletconntest/fakequeryservice.go @@ -698,6 +698,11 @@ func (f *FakeQueryService) VStreamRows(ctx context.Context, request *binlogdatap panic("not implemented") } +// VStreamTables is part of the QueryService interface. +func (f *FakeQueryService) VStreamTables(ctx context.Context, request *binlogdatapb.VStreamTablesRequest, send func(*binlogdatapb.VStreamTablesResponse) error) error { + panic("not implemented") +} + // VStreamResults is part of the QueryService interface. func (f *FakeQueryService) VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { panic("not implemented") diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 2d7d49f2981..94e4741eeee 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -88,7 +88,6 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor ct.sourceTablet.Store(&topodatapb.TabletAlias{}) log.Infof("creating controller with cell: %v, tabletTypes: %v, and params: %v", cell, tabletTypesStr, params) - // id id, err := strconv.ParseInt(params["id"], 10, 32) if err != nil { return nil, err @@ -99,6 +98,10 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor state := params["state"] blpStats.State.Store(state) + if err := prototext.Unmarshal([]byte(params["source"]), ct.source); err != nil { + return nil, err + } + // Nothing to do if replication is stopped or is known to have an unrecoverable error. if state == binlogdatapb.VReplicationWorkflowState_Stopped.String() || state == binlogdatapb.VReplicationWorkflowState_Error.String() { ct.cancel = func() {} @@ -107,14 +110,9 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor return ct, nil } - // source, stopPos - if err := prototext.Unmarshal([]byte(params["source"]), ct.source); err != nil { - return nil, err - } ct.stopPos = params["stop_pos"] if ct.source.GetExternalMysql() == "" { - // tabletPicker if v := params["cell"]; v != "" { cell = v } @@ -138,7 +136,6 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor ct.tabletPicker = tp } - // cancel ctx, ct.cancel = context.WithCancel(ctx) go ct.run(ctx) @@ -167,7 +164,7 @@ func (ct *controller) run(ctx context.Context) { } ct.blpStats.ErrorCounts.Add([]string{"Stream Error"}, 1) - binlogplayer.LogError(fmt.Sprintf("error in stream %v, retrying after %v", ct.id, retryDelay), err) + binlogplayer.LogError(fmt.Sprintf("error in stream %v, will retry after %v", ct.id, retryDelay), err) timer := time.NewTimer(retryDelay) select { case <-ctx.Done(): @@ -257,7 +254,10 @@ func (ct *controller) runBlp(ctx context.Context) (err error) { // If this is a mysql error that we know needs manual intervention OR // we cannot identify this as non-recoverable, but it has persisted // beyond the retry limit (maxTimeToRetryError). - if isUnrecoverableError(err) || !ct.lastWorkflowError.ShouldRetry() { + // In addition, we cannot restart a workflow started with AtomicCopy which has _any_ error. + if (err != nil && vr.WorkflowSubType == int32(binlogdatapb.VReplicationWorkflowSubType_AtomicCopy)) || + isUnrecoverableError(err) || !ct.lastWorkflowError.ShouldRetry() { + log.Errorf("vreplication stream %d going into error state due to %+v", ct.id, err) if errSetState := vr.setState(binlogdatapb.VReplicationWorkflowState_Error, err.Error()); errSetState != nil { log.Errorf("INTERNAL: unable to setState() in controller. Attempting to set error text: [%v]; setState() error is: %v", err, errSetState) diff --git a/go/vt/vttablet/tabletmanager/vreplication/external_connector.go b/go/vt/vttablet/tabletmanager/vreplication/external_connector.go index 5bec2b4f78a..1c20e2054be 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/external_connector.go +++ b/go/vt/vttablet/tabletmanager/vreplication/external_connector.go @@ -52,6 +52,9 @@ type VStreamerClient interface { // VStreamRows streams rows of a table from the specified starting point. VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error + + // VStreamTables streams rows of a table from the specified starting point. + VStreamTables(ctx context.Context, send func(*binlogdatapb.VStreamTablesResponse) error) error } type externalConnector struct { @@ -142,6 +145,10 @@ func (c *mysqlConnector) VStreamRows(ctx context.Context, query string, lastpk * return c.vstreamer.StreamRows(ctx, query, row, send) } +func (c *mysqlConnector) VStreamTables(ctx context.Context, send func(response *binlogdatapb.VStreamTablesResponse) error) error { + return c.vstreamer.StreamTables(ctx, send) +} + //----------------------------------------------------------- type tabletConnector struct { @@ -180,3 +187,8 @@ func (tc *tabletConnector) VStreamRows(ctx context.Context, query string, lastpk req := &binlogdatapb.VStreamRowsRequest{Target: tc.target, Query: query, Lastpk: lastpk} return tc.qs.VStreamRows(ctx, req, send) } + +func (tc *tabletConnector) VStreamTables(ctx context.Context, send func(*binlogdatapb.VStreamTablesResponse) error) error { + req := &binlogdatapb.VStreamTablesRequest{Target: tc.target} + return tc.qs.VStreamTables(ctx, req, send) +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index cbbc564199c..701f5706b9b 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -62,17 +62,16 @@ import ( ) var ( - playerEngine *Engine - streamerEngine *vstreamer.Engine - - envMu sync.Mutex - env *testenv.Env - - globalFBC = &fakeBinlogClient{} - vrepldb = "vrepl" - globalDBQueries = make(chan string, 1000) - testForeignKeyQueries = false - doNotLogDBQueries = false + playerEngine *Engine + streamerEngine *vstreamer.Engine + env *testenv.Env + envMu sync.Mutex + globalFBC = &fakeBinlogClient{} + vrepldb = "vrepl" + globalDBQueries = make(chan string, 1000) + testForeignKeyQueries = false + testSetForeignKeyQueries = false + doNotLogDBQueries = false ) type LogExpectation struct { @@ -477,6 +476,8 @@ func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Resu } if !strings.HasPrefix(query, "select") && !strings.HasPrefix(query, "set") && !dbc.nolog { globalDBQueries <- query + } else if testSetForeignKeyQueries && strings.Contains(query, "set foreign_key_checks") { + globalDBQueries <- query } else if testForeignKeyQueries && strings.Contains(query, "foreign_key_checks") { //allow select/set for foreign_key_checks globalDBQueries <- query } @@ -485,6 +486,9 @@ func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Resu func expectDeleteQueries(t *testing.T) { t.Helper() + if doNotLogDBQueries { + return + } expectNontxQueries(t, qh.Expect( "/delete from _vt.vreplication", "/delete from _vt.copy_state", @@ -549,6 +553,9 @@ func shouldIgnoreQuery(query string) bool { func expectDBClientQueries(t *testing.T, expectations qh.ExpectationSequence, skippableOnce ...string) { t.Helper() + if doNotLogDBQueries { + return + } failed := false skippedOnce := false validator := qh.NewVerifier(expectations) @@ -609,7 +616,9 @@ func expectDBClientQueries(t *testing.T, expectations qh.ExpectationSequence, sk // It also disregards updates to _vt.vreplication. func expectNontxQueries(t *testing.T, expectations qh.ExpectationSequence) { t.Helper() - + if doNotLogDBQueries { + return + } failed := false validator := qh.NewVerifier(expectations) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go index e5db95a795a..cb9c7100522 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -406,7 +406,7 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma copyStateGCTicker := time.NewTicker(copyStateGCInterval) defer copyStateGCTicker.Stop() - parallelism := int(math.Max(1, float64(vreplicationParallelInsertWorkers))) + parallelism := getInsertParallelism() copyWorkerFactory := vc.newCopyWorkerFactory(parallelism) copyWorkQueue := vc.newCopyWorkQueue(parallelism, copyWorkerFactory) defer copyWorkQueue.close() @@ -671,6 +671,16 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma return nil } +func (vc *vcopier) updatePos(ctx context.Context, gtid string) error { + pos, err := replication.DecodePosition(gtid) + if err != nil { + return err + } + update := binlogplayer.GenerateUpdatePos(vc.vr.id, pos, time.Now().Unix(), 0, vc.vr.stats.CopyRowCount.Get(), vreplicationStoreCompressedGTID) + _, err = vc.vr.dbClient.Execute(update) + return err +} + func (vc *vcopier) fastForward(ctx context.Context, copyState map[string]*sqltypes.Result, gtid string) error { defer vc.vr.stats.PhaseTimings.Record("fastforward", time.Now()) pos, err := replication.DecodePosition(gtid) @@ -1074,6 +1084,10 @@ func (vbc *vcopierCopyWorker) execute(ctx context.Context, task *vcopierCopyTask } case vcopierCopyTaskInsertCopyState: advanceFn = func(ctx context.Context, args *vcopierCopyTaskArgs) error { + if vbc.copyStateInsert == nil { + log.Infof("Skipping copy_state insert") + return nil + } if err := vbc.insertCopyState(ctx, args.lastpk); err != nil { return vterrors.Wrapf(err, "error updating _vt.copy_state") } @@ -1200,3 +1214,8 @@ func vcopierCopyTaskGetNextState(vts vcopierCopyTaskState) vcopierCopyTaskState } return vts } + +func getInsertParallelism() int { + parallelism := int(math.Max(1, float64(vreplicationParallelInsertWorkers))) + return parallelism +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go new file mode 100644 index 00000000000..7a9df98d000 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go @@ -0,0 +1,297 @@ +/* +Copyright 2023 The Vitess Authors. + +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 vreplication + +import ( + "context" + "fmt" + "io" + "strconv" + "time" + + "google.golang.org/protobuf/encoding/prototext" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vterrors" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" +) + +/* +This file is similar to vcopier.go: it handles the copy phase for the AtomicCopy where all tables +are streamed in a single phase. +*/ + +type copyAllState struct { + vc *vcopier + plan *ReplicatorPlan + currentTableName string + tables map[string]bool +} + +// newCopyAllState creates the required table plans and sets up the copy state for all tables in the source. +func newCopyAllState(vc *vcopier) (*copyAllState, error) { + state := ©AllState{ + vc: vc, + } + plan, err := buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats) + if err != nil { + return nil, err + } + state.plan = plan + state.tables = make(map[string]bool, len(plan.TargetTables)) + for _, table := range plan.TargetTables { + state.tables[table.TargetName] = false + } + return state, nil +} + +// copyAll copies all tables from the source to the target sequentially, finishing one table first and then moving to the next.. +func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings) error { + var err error + + log.Infof("Starting copyAll for %s", settings.WorkflowName) + defer log.Infof("Returning from copyAll for %s", settings.WorkflowName) + defer vc.vr.dbClient.Rollback() + + state, err := newCopyAllState(vc) + if err != nil { + return err + } + + ctx, cancel := context.WithTimeout(ctx, copyPhaseDuration) + defer cancel() + + rowsCopiedTicker := time.NewTicker(rowsCopiedUpdateInterval) + defer rowsCopiedTicker.Stop() + copyStateGCTicker := time.NewTicker(copyStateGCInterval) + defer copyStateGCTicker.Stop() + + parallelism := getInsertParallelism() + copyWorkerFactory := vc.newCopyWorkerFactory(parallelism) + var copyWorkQueue *vcopierCopyWorkQueue + + // Allocate a result channel to collect results from tasks. + resultCh := make(chan *vcopierCopyTaskResult, parallelism*4) + defer close(resultCh) + + var lastpk *querypb.Row + var pkfields []*querypb.Field + var lastpkbv map[string]*querypb.BindVariable + // Use this for task sequencing. + var prevCh <-chan *vcopierCopyTaskResult + var gtid string + + serr := vc.vr.sourceVStreamer.VStreamTables(ctx, func(resp *binlogdatapb.VStreamTablesResponse) error { + defer vc.vr.stats.PhaseTimings.Record("copy", time.Now()) + defer vc.vr.stats.CopyLoopCount.Add(1) + log.Infof("VStreamTablesResponse: received table %s, #fields %d, #rows %d, gtid %s, lastpk %+v", + resp.TableName, len(resp.Fields), len(resp.Rows), resp.Gtid, resp.Lastpk) + tableName := resp.TableName + gtid = resp.Gtid + + updateRowsCopied := func() error { + updateRowsQuery := binlogplayer.GenerateUpdateRowsCopied(vc.vr.id, vc.vr.stats.CopyRowCount.Get()) + _, err := vc.vr.dbClient.Execute(updateRowsQuery) + return err + } + + if err := updateRowsCopied(); err != nil { + return err + } + select { + case <-rowsCopiedTicker.C: + if err := updateRowsCopied(); err != nil { + return err + } + case <-ctx.Done(): + return io.EOF + default: + } + if tableName != state.currentTableName { + if copyWorkQueue != nil { + copyWorkQueue.close() + } + copyWorkQueue = vc.newCopyWorkQueue(parallelism, copyWorkerFactory) + if state.currentTableName != "" { + log.Infof("copy of table %s is done at lastpk %+v", state.currentTableName, lastpkbv) + if err := vc.deleteCopyState(state.currentTableName); err != nil { + return err + } + } else { + log.Infof("starting copy phase with table %s", tableName) + } + + state.currentTableName = tableName + } + if !copyWorkQueue.isOpen { + if len(resp.Fields) == 0 { + return fmt.Errorf("expecting field event first, got: %v", resp) + } + fieldEvent := &binlogdatapb.FieldEvent{ + TableName: tableName, + } + fieldEvent.Fields = append(fieldEvent.Fields, resp.Fields...) + tablePlan, err := state.plan.buildExecutionPlan(fieldEvent) + if err != nil { + return err + } + pkfields = append(pkfields, pkfields...) + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf( + "insert into _vt.copy_state (lastpk, vrepl_id, table_name) values (%a, %s, %s)", ":lastpk", + strconv.Itoa(int(vc.vr.id)), + encodeString(tableName)) + addLatestCopyState := buf.ParsedQuery() + copyWorkQueue.open(addLatestCopyState, pkfields, tablePlan) + } + if len(resp.Rows) == 0 { + return nil + } + // Get the last committed pk into a loggable form. + lastpkbuf, merr := prototext.Marshal(&querypb.QueryResult{ + Fields: pkfields, + Rows: []*querypb.Row{lastpk}, + }) + if merr != nil { + return fmt.Errorf("failed to marshal pk fields and value into query result: %s", merr.Error()) + } + lastpkbv = map[string]*querypb.BindVariable{ + "lastpk": { + Type: sqltypes.VarBinary, + Value: lastpkbuf, + }, + } + log.Infof("copying table %s with lastpk %v", tableName, lastpkbv) + + // Prepare a vcopierCopyTask for the current batch of work. + currCh := make(chan *vcopierCopyTaskResult, 1) + currT := newVCopierCopyTask(newVCopierCopyTaskArgs(resp.Rows, resp.Lastpk)) + + // Send result to the global resultCh and currCh. resultCh is used by + // the loop to return results to VStreamRows. currCh will be used to + // sequence the start of the nextT. + currT.lifecycle.onResult().sendTo(currCh) + currT.lifecycle.onResult().sendTo(resultCh) + + // Use prevCh to Sequence the prevT with the currT so that: + // * The prevT is completed before we begin updating + // _vt.copy_state for currT. + // * If prevT fails or is canceled, the current task is + // canceled. + // prevCh is nil only for the first task in the vcopier run. + if prevCh != nil { + // prevT publishes to prevCh, and currT is the only thing that can + // consume from prevCh. If prevT is already done, then prevCh will + // have a value in it. If prevT isn't yet done, then prevCh will + // have a value later. Either way, AwaitCompletion should + // eventually get a value, unless there is a context expiry. + currT.lifecycle.before(vcopierCopyTaskInsertCopyState).awaitCompletion(prevCh) + } + + // Store currCh in prevCh. The nextT will use this for sequencing. + prevCh = currCh + + // Update stats after task is done. + currT.lifecycle.onResult().do(func(_ context.Context, result *vcopierCopyTaskResult) { + if result.state == vcopierCopyTaskFail { + vc.vr.stats.ErrorCounts.Add([]string{"Copy"}, 1) + } + if result.state == vcopierCopyTaskComplete { + vc.vr.stats.CopyRowCount.Add(int64(len(result.args.rows))) + vc.vr.stats.QueryCount.Add("copy", 1) + vc.vr.stats.TableCopyRowCounts.Add(tableName, int64(len(result.args.rows))) + vc.vr.stats.TableCopyTimings.Add(tableName, time.Since(result.startedAt)) + } + }) + + if err := copyWorkQueue.enqueue(ctx, currT); err != nil { + log.Warningf("failed to enqueue task in workflow %s: %s", vc.vr.WorkflowName, err.Error()) + return err + } + + // When async execution is not enabled, a done task will be available + // in the resultCh after each Enqueue, unless there was a queue state + // error (e.g. couldn't obtain a worker from pool). + // + // When async execution is enabled, results will show up in the channel + // eventually, possibly in a subsequent VStreamRows loop. It's still + // a good idea to check this channel on every pass so that: + // + // * resultCh doesn't fill up. If it does fill up then tasks won't be + // able to add their results to the channel, and progress in this + // goroutine will be blocked. + // * We keep lastpk up-to-date. + select { + case result := <-resultCh: + if result != nil { + switch result.state { + case vcopierCopyTaskCancel: + log.Warningf("task was canceled in workflow %s: %v", vc.vr.WorkflowName, result.err) + return io.EOF + case vcopierCopyTaskComplete: + // Collect lastpk. Needed for logging at the end. + lastpk = result.args.lastpk + case vcopierCopyTaskFail: + return vterrors.Wrapf(result.err, "task error") + } + } else { + return io.EOF + } + default: + } + return nil + }) + if serr != nil { + log.Infof("VStreamTables failed: %v", serr) + return serr + } + // A context expiration was probably caused by a PlannedReparentShard or an + // elapsed copy phase duration. CopyAll is not resilient to these events. + select { + case <-ctx.Done(): + log.Infof("Copy of %v stopped", state.currentTableName) + return fmt.Errorf("CopyAll was interrupted due to context expiration") + default: + if err := vc.deleteCopyState(state.currentTableName); err != nil { + return err + } + if copyWorkQueue != nil { + copyWorkQueue.close() + } + if err := vc.updatePos(ctx, gtid); err != nil { + return err + } + log.Infof("Completed copy of all tables") + } + return nil +} + +// deleteCopyState deletes the copy state entry for a table, signifying that the copy phase is complete for that table. +func (vc *vcopier) deleteCopyState(tableName string) error { + log.Infof("Deleting copy state for table %s", tableName) + //FIXME get sidecar db name + delQuery := fmt.Sprintf("delete from _vt.copy_state where table_name=%s and vrepl_id = %d", encodeString(tableName), vc.vr.id) + if _, err := vc.vr.dbClient.Execute(delQuery); err != nil { + return err + } + return nil +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go index 129827c5d47..ac259294419 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go @@ -483,13 +483,13 @@ func testPlayerCopyTablesWithFK(t *testing.T) { "/insert into _vt.copy_state", "/update _vt.vreplication set state='Copying'", "commit", - "set foreign_key_checks=0", + "set @@session.foreign_key_checks=0", // The first fast-forward has no starting point. So, it just saves the current position. "/update _vt.vreplication set pos=", ).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer { // With parallel inserts, new db client connects are created on-the-fly. if vreplicationParallelInsertWorkers > 1 { - return expect.Then(qh.Eventually("set foreign_key_checks=0")) + return expect.Then(qh.Eventually("set @@session.foreign_key_checks=0")) } return expect }).Then(qh.Eventually( @@ -500,18 +500,18 @@ func testPlayerCopyTablesWithFK(t *testing.T) { `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:\\"id\\" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:\\"2\\"}'.*`, "commit", )).Then(qh.Immediately( - "set foreign_key_checks=0", + "set @@session.foreign_key_checks=0", // copy of dst1 is done: delete from copy_state. "/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst1", // The next FF executes and updates the position before copying. - "set foreign_key_checks=0", + "set @@session.foreign_key_checks=0", "begin", "/update _vt.vreplication set pos=", "commit", )).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer { // With parallel inserts, new db client connects are created on-the-fly. if vreplicationParallelInsertWorkers > 1 { - return expect.Then(qh.Eventually("set foreign_key_checks=0")) + return expect.Then(qh.Eventually("set @@session.foreign_key_checks=0")) } return expect }).Then(qh.Eventually( @@ -521,11 +521,11 @@ func testPlayerCopyTablesWithFK(t *testing.T) { `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:\\"id\\" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:\\"2\\"}'.*`, "commit", )).Then(qh.Immediately( - "set foreign_key_checks=0", + "set @@session.foreign_key_checks=0", // copy of dst1 is done: delete from copy_state. "/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst2", // All tables copied. Final catch up followed by Running state. - "set foreign_key_checks=1", + "set @@session.foreign_key_checks=1", "/update _vt.vreplication set state='Running'", ))) @@ -545,7 +545,7 @@ func testPlayerCopyTablesWithFK(t *testing.T) { t.Fatal(err) } expectDBClientQueries(t, qh.Expect( - "set foreign_key_checks=1", + "set @@session.foreign_key_checks=1", "begin", "/delete from _vt.vreplication", "/delete from _vt.copy_state", diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 998fca634a7..e0ad2eb07bc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -68,8 +68,12 @@ type vplayer struct { phase string throttlerAppName string + + foreignKeyCheck bool } +const NoForeignKeyCheckFlagBitmask uint32 = 1 << 1 + // newVPlayer creates a new vplayer. Parameters: // vreplicator: the outer replicator. It's used for common functions like setState. // @@ -101,6 +105,7 @@ func newVPlayer(vr *vreplicator, settings binlogplayer.VRSettings, copyState map tablePlans: make(map[string]*TablePlan), phase: phase, throttlerAppName: throttlerapp.VCopierName.ConcatenateString(vr.throttlerAppName()), + foreignKeyCheck: false, } } @@ -133,6 +138,25 @@ func (vp *vplayer) play(ctx context.Context) error { return vp.fetchAndApply(ctx) } +// updateFKCheck will check if the fk checks value has changed from what it is currently set to and update it if it has. +// This is done to avoid setting the fk checks value for every row event. vplayer starts with fk checks off. +func (vp *vplayer) updateFKCheck(ctx context.Context, flags2 uint32) error { + // The 2nd bit (least significant) of the flags attribute of the binlog row event + // is set to 1 if foreign key checks are disabled. + enableFK := true + if flags2&NoForeignKeyCheckFlagBitmask == NoForeignKeyCheckFlagBitmask { + enableFK = false + } + if vp.foreignKeyCheck == enableFK { + // if not changed, return + return nil + } + vp.foreignKeyCheck = enableFK + log.Infof("Setting foreign_key_checks to %v", enableFK) + _, err := vp.vr.dbClient.ExecuteWithRetry(ctx, "set @@session.foreign_key_checks="+strconv.FormatBool(enableFK)) + return err +} + // fetchAndApply performs the fetching and application of the binlogs. // This is done by two different threads. The fetcher thread pulls // events from the vstreamer and adds them to the relayLog. @@ -217,6 +241,9 @@ func (vp *vplayer) applyStmtEvent(ctx context.Context, event *binlogdatapb.VEven } func (vp *vplayer) applyRowEvent(ctx context.Context, rowEvent *binlogdatapb.RowEvent) error { + if err := vp.updateFKCheck(ctx, rowEvent.Flags); err != nil { + return err + } tplan := vp.tablePlans[rowEvent.TableName] if tplan == nil { return fmt.Errorf("unexpected event on table %s", rowEvent.TableName) @@ -503,6 +530,7 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m return err } if err := vp.applyRowEvent(ctx, event.RowEvent); err != nil { + log.Infof("Error applying row event: %s", err.Error()) return err } //Row event is logged AFTER RowChanges are applied so as to calculate the total elapsed time for the Row event diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go index 2aa5697df63..f7838f2e6a2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go @@ -528,6 +528,62 @@ func TestPlayerSavepoint(t *testing.T) { cancel() } +// TestPlayerForeignKeyCheck tests that we can insert a row into a child table without the corresponding foreign key +// if the foreign_key_checks is not set. +func TestPlayerForeignKeyCheck(t *testing.T) { + doNotLogDBQueries = true + defer func() { doNotLogDBQueries = false }() + + defer deleteTablet(addTablet(100)) + execStatements(t, []string{ + "create table parent(id int, name varchar(128), primary key(id))", + fmt.Sprintf("create table %s.parent(id int, name varchar(128), primary key(id))", vrepldb), + "create table child(id int, parent_id int, name varchar(128), primary key(id), foreign key(parent_id) references parent(id) on delete cascade)", + fmt.Sprintf("create table %s.child(id int, parent_id int, name varchar(128), primary key(id), foreign key(parent_id) references parent(id) on delete cascade)", vrepldb), + }) + defer execStatements(t, []string{ + "drop table child", + fmt.Sprintf("drop table %s.child", vrepldb), + "drop table parent", + fmt.Sprintf("drop table %s.parent", vrepldb), + }) + + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: filter, + OnDdl: binlogdatapb.OnDDLAction_IGNORE, + } + cancel, _ := startVReplication(t, bls, "") + + testSetForeignKeyQueries = true + defer func() { + testSetForeignKeyQueries = false + }() + + execStatements(t, []string{ + "insert into parent values(1, 'parent1')", + "insert into child values(1, 1, 'child1')", + "set foreign_key_checks=0", + "insert into child values(2, 100, 'child100')", + }) + expectData(t, "parent", [][]string{ + {"1", "parent1"}, + }) + expectData(t, "child", [][]string{ + {"1", "1", "child1"}, + {"2", "100", "child100"}, + }) + cancel() +} + func TestPlayerStatementModeWithFilter(t *testing.T) { defer deleteTablet(addTablet(100)) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index 39bd8db2519..c8d04c80294 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -108,8 +108,9 @@ type vreplicator struct { originalFKCheckSetting int64 originalSQLMode string - WorkflowType int32 - WorkflowName string + WorkflowType int32 + WorkflowSubType int32 + WorkflowName string throttleUpdatesRateLimiter *timer.RateLimiter } @@ -271,18 +272,26 @@ func (vr *vreplicator) replicate(ctx context.Context) error { log.Warningf("Unable to clear FK check %v", err) return err } - if err := newVCopier(vr).copyNext(ctx, settings); err != nil { - vr.stats.ErrorCounts.Add([]string{"Copy"}, 1) - return err - } - settings, numTablesToCopy, err = vr.loadSettings(ctx, vr.dbClient) - if err != nil { - return err - } - if numTablesToCopy == 0 { - if err := vr.insertLog(LogCopyEnd, fmt.Sprintf("Copy phase completed at gtid %s", settings.StartPos)); err != nil { + if vr.WorkflowSubType == int32(binlogdatapb.VReplicationWorkflowSubType_AtomicCopy) { + if err := newVCopier(vr).copyAll(ctx, settings); err != nil { + log.Infof("Error copying all tables: %v", err) + vr.stats.ErrorCounts.Add([]string{"CopyAll"}, 1) + return err + } + } else { + if err := newVCopier(vr).copyNext(ctx, settings); err != nil { + vr.stats.ErrorCounts.Add([]string{"Copy"}, 1) return err } + settings, numTablesToCopy, err = vr.loadSettings(ctx, vr.dbClient) + if err != nil { + return err + } + if numTablesToCopy == 0 { + if err := vr.insertLog(LogCopyEnd, fmt.Sprintf("Copy phase completed at gtid %s", settings.StartPos)); err != nil { + return err + } + } } case settings.StartPos.IsZero(): if err := newVCopier(vr).initTablesForCopy(ctx); err != nil { @@ -406,6 +415,7 @@ func (vr *vreplicator) loadSettings(ctx context.Context, dbClient *vdbClient) (s settings, numTablesToCopy, err = vr.readSettings(ctx, dbClient) if err == nil { vr.WorkflowType = int32(settings.WorkflowType) + vr.WorkflowSubType = int32(settings.WorkflowSubType) vr.WorkflowName = settings.WorkflowName } return settings, numTablesToCopy, err @@ -499,7 +509,7 @@ func (vr *vreplicator) getSettingFKCheck() error { } func (vr *vreplicator) resetFKCheckAfterCopy(dbClient *vdbClient) error { - _, err := dbClient.Execute(fmt.Sprintf("set foreign_key_checks=%d", vr.originalFKCheckSetting)) + _, err := dbClient.Execute(fmt.Sprintf("set @@session.foreign_key_checks=%d", vr.originalFKCheckSetting)) return err } @@ -587,7 +597,7 @@ func (vr *vreplicator) updateHeartbeatTime(tm int64) error { } func (vr *vreplicator) clearFKCheck(dbClient *vdbClient) error { - _, err := dbClient.Execute("set foreign_key_checks=0") + _, err := dbClient.Execute("set @@session.foreign_key_checks=0") return err } diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 497b368d1fc..8ebc2770fa3 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1163,6 +1163,14 @@ func (tsv *TabletServer) VStreamRows(ctx context.Context, request *binlogdatapb. return tsv.vstreamer.StreamRows(ctx, request.Query, row, send) } +// VStreamTables streams all tables. +func (tsv *TabletServer) VStreamTables(ctx context.Context, request *binlogdatapb.VStreamTablesRequest, send func(*binlogdatapb.VStreamTablesResponse) error) error { + if err := tsv.sm.VerifyTarget(ctx, request.Target); err != nil { + return err + } + return tsv.vstreamer.StreamTables(ctx, send) +} + // VStreamResults streams rows from the specified starting point. func (tsv *TabletServer) VStreamResults(ctx context.Context, target *querypb.Target, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { if err := tsv.sm.VerifyTarget(ctx, target); err != nil { diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go index c55c312f442..adbd117c2f2 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -70,6 +70,7 @@ type Engine struct { streamIdx int streamers map[int]*uvstreamer rowStreamers map[int]*rowStreamer + tableStreamers map[int]*tableStreamer resultStreamers map[int]*resultStreamer // watcherOnce is used for initializing vschema @@ -99,6 +100,7 @@ type Engine struct { vstreamersCreated *stats.Counter vstreamersEndedWithErrors *stats.Counter vstreamerFlushedBinlogs *stats.Counter + tableStreamerNumTables *stats.Counter throttlerClient *throttle.Client } @@ -116,6 +118,7 @@ func NewEngine(env tabletenv.Env, ts srvtopo.Server, se *schema.Engine, lagThrot streamers: make(map[int]*uvstreamer), rowStreamers: make(map[int]*rowStreamer), + tableStreamers: make(map[int]*tableStreamer), resultStreamers: make(map[int]*resultStreamer), lvschema: &localVSchema{vschema: &vindexes.VSchema{}}, @@ -135,6 +138,7 @@ func NewEngine(env tabletenv.Env, ts srvtopo.Server, se *schema.Engine, lagThrot rowStreamerNumRows: env.Exporter().NewCounter("RowStreamerNumRows", "Number of rows sent in row streamer"), rowStreamerWaits: env.Exporter().NewTimings("RowStreamerWaits", "Total counts and time we've waited when streaming rows in the vstream copy phase", "copy-phase-waits"), vstreamersCreated: env.Exporter().NewCounter("VStreamersCreated", "Count of vstreamers created"), + tableStreamerNumTables: env.Exporter().NewCounter("TableStreamerNumTables", "Number of tables streamed by the table streamer"), vstreamersEndedWithErrors: env.Exporter().NewCounter("VStreamersEndedWithErrors", "Count of vstreamers that ended with errors"), errorCounts: env.Exporter().NewCountersWithSingleLabel("VStreamerErrors", "Tracks errors in vstreamer", "type", "Catchup", "Copy", "Send", "TablePlan"), vstreamerFlushedBinlogs: env.Exporter().NewCounter("VStreamerFlushedBinlogs", "Number of times we've successfully executed a FLUSH BINARY LOGS statement when starting a vstream"), @@ -283,7 +287,7 @@ func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltyp vse.mu.Lock() defer vse.mu.Unlock() - rowStreamer := newRowStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, query, lastpk, vse.lvschema, send, vse) + rowStreamer := newRowStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, query, lastpk, vse.lvschema, send, vse, RowStreamerModeSingleTable, nil) idx := vse.streamIdx vse.rowStreamers[idx] = rowStreamer vse.streamIdx++ @@ -308,6 +312,47 @@ func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltyp return rowStreamer.Stream() } +// StreamTables streams all tables. +func (vse *Engine) StreamTables(ctx context.Context, send func(*binlogdatapb.VStreamTablesResponse) error) error { + // Ensure vschema is initialized and the watcher is started. + // Starting of the watcher is delayed till the first call to StreamTables + // so that this overhead is incurred only if someone uses this feature. + vse.watcherOnce.Do(vse.setWatch) + log.Infof("Streaming all tables") + + // Create stream and add it to the map. + tableStreamer, idx, err := func() (*tableStreamer, int, error) { + if atomic.LoadInt32(&vse.isOpen) == 0 { + return nil, 0, errors.New("VStreamer is not open") + } + vse.mu.Lock() + defer vse.mu.Unlock() + + tableStreamer := newTableStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, vse.lvschema, send, vse) + idx := vse.streamIdx + vse.tableStreamers[idx] = tableStreamer + vse.streamIdx++ + // Now that we've added the stream, increment wg. + // This must be done before releasing the lock. + vse.wg.Add(1) + return tableStreamer, idx, nil + }() + if err != nil { + return err + } + + // Remove stream from map and decrement wg when it ends. + defer func() { + vse.mu.Lock() + defer vse.mu.Unlock() + delete(vse.tableStreamers, idx) + vse.wg.Done() + }() + + // No lock is held while streaming, but wg is incremented. + return tableStreamer.Stream() +} + // StreamResults streams results of the query with the gtid. func (vse *Engine) StreamResults(ctx context.Context, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error { // Create stream and add it to the map. diff --git a/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go index 442c998b71c..f3743c6de46 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go @@ -36,6 +36,7 @@ var ( env *testenv.Env ignoreKeyspaceShardInFieldAndRowEvents bool + testRowEventFlags bool ) func TestMain(m *testing.M) { diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go index 86b50947cdf..00194b5cc0b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -50,10 +50,17 @@ type RowStreamer interface { } // NewRowStreamer returns a RowStreamer -func NewRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, query string, lastpk []sqltypes.Value, send func(*binlogdatapb.VStreamRowsResponse) error, vse *Engine) RowStreamer { - return newRowStreamer(ctx, cp, se, query, lastpk, &localVSchema{vschema: &vindexes.VSchema{}}, send, vse) +func NewRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, query string, lastpk []sqltypes.Value, send func(*binlogdatapb.VStreamRowsResponse) error, vse *Engine, mode RowStreamerMode) RowStreamer { + return newRowStreamer(ctx, cp, se, query, lastpk, &localVSchema{vschema: &vindexes.VSchema{}}, send, vse, mode, nil) } +type RowStreamerMode int32 + +const ( + RowStreamerModeSingleTable RowStreamerMode = iota + RowStreamerModeAllTables +) + // rowStreamer is used for copying the existing rows of a table // before vreplication begins streaming binlogs. The rowStreamer // responds to a request with the GTID position as of which it @@ -79,9 +86,15 @@ type rowStreamer struct { sendQuery string vse *Engine pktsize PacketSizer + + mode RowStreamerMode + conn *snapshotConn } -func newRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, query string, lastpk []sqltypes.Value, vschema *localVSchema, send func(*binlogdatapb.VStreamRowsResponse) error, vse *Engine) *rowStreamer { +func newRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, query string, + lastpk []sqltypes.Value, vschema *localVSchema, send func(*binlogdatapb.VStreamRowsResponse) error, vse *Engine, + mode RowStreamerMode, conn *snapshotConn) *rowStreamer { + ctx, cancel := context.WithCancel(ctx) return &rowStreamer{ ctx: ctx, @@ -94,6 +107,8 @@ func newRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engi vschema: vschema, vse: vse, pktsize: DefaultPacketSizer(), + mode: mode, + conn: conn, } } @@ -111,15 +126,18 @@ func (rs *rowStreamer) Stream() error { if err := rs.buildPlan(); err != nil { return err } - conn, err := snapshotConnect(rs.ctx, rs.cp) - if err != nil { - return err - } - defer conn.Close() - if _, err := conn.ExecuteFetch("set names 'binary'", 1, false); err != nil { - return err + if rs.conn == nil { + conn, err := snapshotConnect(rs.ctx, rs.cp) + if err != nil { + return err + } + rs.conn = conn + defer rs.conn.Close() + if _, err := rs.conn.ExecuteFetch("set names 'binary'", 1, false); err != nil { + return err + } } - return rs.streamQuery(conn, rs.send) + return rs.streamQuery(rs.send) } func (rs *rowStreamer) buildPlan() error { @@ -279,7 +297,7 @@ func (rs *rowStreamer) buildSelect() (string, error) { return buf.String(), nil } -func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.VStreamRowsResponse) error) error { +func (rs *rowStreamer) streamQuery(send func(*binlogdatapb.VStreamRowsResponse) error) error { throttleResponseRateLimiter := timer.NewRateLimiter(rowStreamertHeartbeatInterval) defer throttleResponseRateLimiter.Stop() @@ -293,14 +311,25 @@ func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.V if err := rs.vse.waitForMySQL(rs.ctx, rs.cp, rs.plan.Table.Name); err != nil { return err } - + var ( + gtid string + rotatedLog bool + err error + ) log.Infof("Streaming query: %v\n", rs.sendQuery) - gtid, rotatedLog, err := conn.streamWithSnapshot(rs.ctx, rs.plan.Table.Name, rs.sendQuery) - if rotatedLog { - rs.vse.vstreamerFlushedBinlogs.Add(1) - } - if err != nil { - return err + if rs.mode == RowStreamerModeSingleTable { + gtid, rotatedLog, err = rs.conn.streamWithSnapshot(rs.ctx, rs.plan.Table.Name, rs.sendQuery) + if err != nil { + return err + } + if rotatedLog { + rs.vse.vstreamerFlushedBinlogs.Add(1) + } + } else { + // Comes here when we stream all tables. The snapshot is created just once at the start. + if err := rs.conn.ExecuteStreamFetch(rs.query); err != nil { + return err + } } pkfields := make([]*querypb.Field, len(rs.pkColumns)) @@ -339,10 +368,12 @@ func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.V } }() - var response binlogdatapb.VStreamRowsResponse - var rows []*querypb.Row - var rowCount int - var mysqlrow []sqltypes.Value + var ( + response binlogdatapb.VStreamRowsResponse + rows []*querypb.Row + rowCount int + mysqlrow []sqltypes.Value + ) filtered := make([]sqltypes.Value, len(rs.plan.ColExprs)) lastpk := make([]sqltypes.Value, len(rs.pkColumns)) @@ -364,7 +395,7 @@ func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.V if mysqlrow != nil { mysqlrow = mysqlrow[:0] } - mysqlrow, err = conn.FetchNext(mysqlrow) + mysqlrow, err = rs.conn.FetchNext(mysqlrow) if err != nil { return err } diff --git a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go index 3b4bb8f3d8c..c3fead67508 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go +++ b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go @@ -217,3 +217,43 @@ func GetBinlogRotationThreshold() int64 { func SetBinlogRotationThreshold(threshold int64) { atomic.StoreInt64(&binlogRotationThreshold, threshold) } + +// startSnapshotAllTables starts a streaming query with a snapshot view of all tables, returning the +// GTID set from the time when the snapshot was taken. +func (conn *snapshotConn) startSnapshotAllTables(ctx context.Context) (gtid string, err error) { + lockConn, err := mysqlConnect(ctx, conn.cp) + if err != nil { + return "", err + } + // To be safe, always unlock tables, even if lock tables might fail. + defer func() { + _, err := lockConn.ExecuteFetch("unlock tables", 0, false) + if err != nil { + log.Warning("Unlock tables failed: %v", err) + } + lockConn.Close() + }() + + log.Infof("Locking all tables") + if _, err := lockConn.ExecuteFetch("FLUSH TABLES WITH READ LOCK", 1, false); err != nil { + log.Infof("Error locking all tables") + return "", err + } + mpos, err := lockConn.PrimaryPosition() + if err != nil { + return "", err + } + + // Starting a transaction now will allow us to start the read later, + // which will happen after we release the lock on the table. + if _, err := conn.ExecuteFetch("set transaction isolation level repeatable read", 1, false); err != nil { + return "", err + } + if _, err := conn.ExecuteFetch("start transaction with consistent snapshot", 1, false); err != nil { + return "", err + } + if _, err := conn.ExecuteFetch("set @@session.time_zone = '+00:00'", 1, false); err != nil { + return "", err + } + return replication.EncodePosition(mpos), nil +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go new file mode 100644 index 00000000000..31f9d23de87 --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go @@ -0,0 +1,176 @@ +/* +Copyright 2023 The Vitess Authors. + +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 vstreamer + +import ( + "context" + "errors" + "fmt" + "sync/atomic" + + "vitess.io/vitess/go/sqlescape" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/dbconfigs" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +/* + TableStreamer is a VStreamer that streams all tables in a keyspace. It iterates through all tables in a keyspace + and streams them one by one. It is not resilient: if there is any error that breaks the stream, for example, + reparenting or a network error, it will not recover and a new workflow will have to be created. +*/ + +// TableStreamer exposes an externally usable interface to tableStreamer. +type TableStreamer interface { + Stream() error + Cancel() +} + +type tableStreamer struct { + ctx context.Context + cancel func() + + cp dbconfigs.Connector + se *schema.Engine + send func(*binlogdatapb.VStreamTablesResponse) error + vschema *localVSchema + vse *Engine + + snapshotConn *snapshotConn + tables []string + gtid string +} + +func newTableStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, vschema *localVSchema, send func(response *binlogdatapb.VStreamTablesResponse) error, vse *Engine) *tableStreamer { + ctx, cancel := context.WithCancel(ctx) + return &tableStreamer{ + ctx: ctx, + cancel: cancel, + cp: cp, + se: se, + send: send, + vschema: vschema, + vse: vse, + } +} + +func (ts *tableStreamer) Cancel() { + log.Info("TableStreamer Cancel() called") + ts.cancel() +} + +func (ts *tableStreamer) Stream() error { + // Ensure that the schema engine is Open. If vttablet came up in a non_serving role, it may not have been initialized. + var err error + if err = ts.se.Open(); err != nil { + return err + } + + conn, err := snapshotConnect(ts.ctx, ts.cp) + if err != nil { + return err + } + defer conn.Close() + ts.snapshotConn = conn + + if ts.gtid, err = conn.startSnapshotAllTables(ts.ctx); err != nil { + return err + } + + if _, err := conn.ExecuteFetch("set names 'binary'", 1, false); err != nil { + return err + } + + rs, err := conn.ExecuteFetch("show tables", -1, true) + if err != nil { + return err + } + for _, row := range rs.Rows { + tableName := row[0].ToString() + ts.tables = append(ts.tables, tableName) + } + log.Infof("Found %d tables to stream: %+v", len(ts.tables), ts.tables) + for _, tableName := range ts.tables { + log.Infof("Streaming table %s", tableName) + if err := ts.streamTable(ts.ctx, tableName); err != nil { + return err + } + log.Infof("Finished streaming table %s", tableName) + } + log.Infof("Finished streaming %d tables", len(ts.tables)) + return nil +} + +func (ts *tableStreamer) newRowStreamer(ctx context.Context, query string, lastpk []sqltypes.Value, + send func(*binlogdatapb.VStreamRowsResponse) error) (*rowStreamer, func(), error) { + + vse := ts.vse + if atomic.LoadInt32(&vse.isOpen) == 0 { + return nil, nil, errors.New("VStreamer is not open") + } + vse.mu.Lock() + defer vse.mu.Unlock() + + rowStreamer := newRowStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, query, lastpk, vse.lvschema, + send, vse, RowStreamerModeAllTables, ts.snapshotConn) + + idx := vse.streamIdx + vse.rowStreamers[idx] = rowStreamer + vse.streamIdx++ + // Now that we've added the stream, increment wg. + // This must be done before releasing the lock. + vse.wg.Add(1) + + // Remove stream from map and decrement wg when it ends. + cancel := func() { + vse.mu.Lock() + defer vse.mu.Unlock() + delete(vse.rowStreamers, idx) + vse.wg.Done() + } + return rowStreamer, cancel, nil +} + +func (ts *tableStreamer) streamTable(ctx context.Context, tableName string) error { + query := fmt.Sprintf("select * from %s", sqlescape.EscapeID(tableName)) + + send := func(response *binlogdatapb.VStreamRowsResponse) error { + return ts.send(&binlogdatapb.VStreamTablesResponse{ + TableName: tableName, + Fields: response.GetFields(), + Pkfields: response.GetPkfields(), + Gtid: ts.gtid, + Rows: response.GetRows(), + Lastpk: response.Lastpk, + }) + } + rs, cancel, err := ts.newRowStreamer(ctx, query, nil, send) + if err != nil { + return err + } + defer cancel() + + if rs.Stream() != nil { + return err + } + rs.vse.tableStreamerNumTables.Add(int64(1)) + + return nil +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go new file mode 100644 index 00000000000..1062e5e4a03 --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go @@ -0,0 +1,78 @@ +/* +Copyright 2023 The Vitess Authors. + +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 vstreamer + +import ( + "context" + "fmt" + "testing" + + "github.com/stretchr/testify/require" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +// TestTableStreamer streams multiple tables and ensures all rows are received in the correct order. +func TestTableStreamer(t *testing.T) { + ctx := context.Background() + execStatements(t, []string{ + // Single PK + "create table t1(id int, val varbinary(128), primary key(id))", + "insert into t1 values (1, 'aaa'), (2, 'bbb')", + // Composite PK + "create table t2(id1 int, id2 int, val varbinary(128), primary key(id1, id2))", + "insert into t2 values (1, 2, 'aaa'), (1, 3, 'bbb')", + // No PK + "create table t3(id int, val varbinary(128))", + "insert into t3 values (1, 'aaa'), (2, 'bbb')", + // Three-column PK + "create table t4(id1 int, id2 int, id3 int, val varbinary(128), primary key(id1, id2, id3))", + "insert into t4 values (1, 2, 3, 'aaa'), (2, 3, 4, 'bbb')", + }) + + defer execStatements(t, []string{ + "drop table t1", + "drop table t2", + "drop table t3", + "drop table t4", + }) + + engine.se.Reload(context.Background()) + + wantStream := []string{ + "table_name:\"t1\" fields:{name:\"id\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id\" column_length:11 charset:63 flags:53251} fields:{name:\"val\" type:VARBINARY table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"val\" column_length:128 charset:63 flags:128} pkfields:{name:\"id\" type:INT32 charset:63 flags:53251}", + "table_name:\"t1\" rows:{lengths:1 lengths:3 values:\"1aaa\"} rows:{lengths:1 lengths:3 values:\"2bbb\"} lastpk:{lengths:1 values:\"2\"}", + "table_name:\"t2\" fields:{name:\"id1\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id1\" column_length:11 charset:63 flags:53251} fields:{name:\"id2\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id2\" column_length:11 charset:63 flags:53251} fields:{name:\"val\" type:VARBINARY table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"val\" column_length:128 charset:63 flags:128} pkfields:{name:\"id1\" type:INT32 charset:63 flags:53251} pkfields:{name:\"id2\" type:INT32 charset:63 flags:53251}", + "table_name:\"t2\" rows:{lengths:1 lengths:1 lengths:3 values:\"12aaa\"} rows:{lengths:1 lengths:1 lengths:3 values:\"13bbb\"} lastpk:{lengths:1 lengths:1 values:\"13\"}", + "table_name:\"t3\" fields:{name:\"id\" type:INT32 table:\"t3\" org_table:\"t3\" database:\"vttest\" org_name:\"id\" column_length:11 charset:63 flags:32768} fields:{name:\"val\" type:VARBINARY table:\"t3\" org_table:\"t3\" database:\"vttest\" org_name:\"val\" column_length:128 charset:63 flags:128} pkfields:{name:\"id\" type:INT32 charset:63 flags:32768} pkfields:{name:\"val\" type:VARBINARY charset:63 flags:128}", + "table_name:\"t3\" rows:{lengths:1 lengths:3 values:\"1aaa\"} rows:{lengths:1 lengths:3 values:\"2bbb\"} lastpk:{lengths:1 lengths:3 values:\"2bbb\"}", + "table_name:\"t4\" fields:{name:\"id1\" type:INT32 table:\"t4\" org_table:\"t4\" database:\"vttest\" org_name:\"id1\" column_length:11 charset:63 flags:53251} fields:{name:\"id2\" type:INT32 table:\"t4\" org_table:\"t4\" database:\"vttest\" org_name:\"id2\" column_length:11 charset:63 flags:53251} fields:{name:\"id3\" type:INT32 table:\"t4\" org_table:\"t4\" database:\"vttest\" org_name:\"id3\" column_length:11 charset:63 flags:53251} fields:{name:\"val\" type:VARBINARY table:\"t4\" org_table:\"t4\" database:\"vttest\" org_name:\"val\" column_length:128 charset:63 flags:128} pkfields:{name:\"id1\" type:INT32 charset:63 flags:53251} pkfields:{name:\"id2\" type:INT32 charset:63 flags:53251} pkfields:{name:\"id3\" type:INT32 charset:63 flags:53251}", + "table_name:\"t4\" rows:{lengths:1 lengths:1 lengths:1 lengths:3 values:\"123aaa\"} rows:{lengths:1 lengths:1 lengths:1 lengths:3 values:\"234bbb\"} lastpk:{lengths:1 lengths:1 lengths:1 values:\"234\"}", + } + var gotStream []string + err := engine.StreamTables(ctx, func(response *binlogdatapb.VStreamTablesResponse) error { + response.Gtid = "" + for _, fld := range response.Fields { + fld.ColumnType = "" + } + gotStream = append(gotStream, fmt.Sprintf("%v", response)) + return nil + }) + require.NoError(t, err) + require.EqualValues(t, wantStream, gotStream) + require.Equal(t, int64(4), engine.tableStreamerNumTables.Get()) +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go index 91505ba14af..34f860d2120 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go @@ -450,6 +450,9 @@ func startVStreamCopy(ctx context.Context, t *testing.T, filter *binlogdatapb.Fi if ev.Type == binlogdatapb.VEventType_HEARTBEAT { continue } + if ev.Type == binlogdatapb.VEventType_ROW { + ev.RowEvent.Flags = 0 + } if ev.Throttled { continue } diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index 0e808f324ff..c538d6d3871 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -462,7 +462,6 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e } // Insert/Delete/Update are supported only to be used in the context of external mysql streams where source databases // could be using SBR. Vitess itself will never run into cases where it needs to consume non rbr statements. - switch cat := sqlparser.Preview(q.SQL); cat { case sqlparser.StmtInsert: mustSend := mustSendStmt(q, vs.cp.DBName()) @@ -944,6 +943,7 @@ func (vs *vstreamer) processRowEvent(vevents []*binlogdatapb.VEvent, plan *strea RowChanges: rowChanges, Keyspace: vs.vse.keyspace, Shard: vs.vse.shard, + Flags: uint32(rows.Flags), }, }) } diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go index fec0c10ded7..0eda0d6c52e 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go @@ -291,6 +291,52 @@ func TestSetStatement(t *testing.T) { runCases(t, nil, testcases, "current", nil) } +// TestSetForeignKeyCheck confirms that the binlog RowEvent flags are set correctly when foreign_key_checks are on and off. +func TestSetForeignKeyCheck(t *testing.T) { + testRowEventFlags = true + defer func() { testRowEventFlags = false }() + + execStatements(t, []string{ + "create table t1(id int, val binary(4), primary key(id))", + }) + defer execStatements(t, []string{ + "drop table t1", + }) + engine.se.Reload(context.Background()) + queries := []string{ + "begin", + "insert into t1 values (1, 'aaa')", + "set @@session.foreign_key_checks=1", + "insert into t1 values (2, 'bbb')", + "set @@session.foreign_key_checks=0", + "insert into t1 values (3, 'ccc')", + "commit", + } + + fe := &TestFieldEvent{ + table: "t1", + db: "vttest", + cols: []*TestColumn{ + {name: "id", dataType: "INT32", colType: "int(11)", len: 11, charset: 63}, + {name: "val", dataType: "BINARY", colType: "binary(4)", len: 4, charset: 63}, + }, + } + + testcases := []testcase{{ + input: queries, + output: [][]string{{ + `begin`, + fe.String(), + `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"1aaa\x00"}} flags:1}`, + `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"2bbb\x00"}} flags:1}`, + `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"3ccc\x00"}} flags:3}`, + `gtid`, + `commit`, + }}, + }} + runCases(t, nil, testcases, "current", nil) +} + func TestStmtComment(t *testing.T) { if testing.Short() { @@ -2294,6 +2340,9 @@ func expectLog(ctx context.Context, t *testing.T, input any, ch <-chan []*binlog evs[i].RowEvent.Keyspace = "" evs[i].RowEvent.Shard = "" } + if !testRowEventFlags && evs[i].Type == binlogdatapb.VEventType_ROW { + evs[i].RowEvent.Flags = 0 + } want = env.RemoveAnyDeprecatedDisplayWidths(want) if got := fmt.Sprintf("%v", evs[i]); got != want { log.Errorf("%v (%d): event:\n%q, want\n%q", input, i, got, want) diff --git a/go/vt/wrangler/materializer.go b/go/vt/wrangler/materializer.go index 41095df6758..b24aea00136 100644 --- a/go/vt/wrangler/materializer.go +++ b/go/vt/wrangler/materializer.go @@ -131,7 +131,7 @@ func shouldInclude(table string, excludes []string) bool { func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, targetKeyspace, tableSpecs, cell, tabletTypesStr string, allTables bool, excludeTables string, autoStart, stopAfterCopy bool, externalCluster string, dropForeignKeys, deferSecondaryKeys bool, sourceTimeZone, onDDL string, - sourceShards []string, noRoutingRules bool) (err error) { + sourceShards []string, noRoutingRules bool, atomicCopy bool) (err error) { //FIXME validate tableSpecs, allTables, excludeTables var tables []string var externalTopo *topo.Server @@ -238,6 +238,7 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta SourceShards: sourceShards, OnDdl: onDDL, DeferSecondaryKeys: deferSecondaryKeys, + AtomicCopy: atomicCopy, } if sourceTimeZone != "" { ms.SourceTimeZone = sourceTimeZone @@ -1386,20 +1387,14 @@ func (mz *materializer) generateInserts(ctx context.Context, targetShard *topo.S bls.Filter.Rules = append(bls.Filter.Rules, rule) } - workflowSubType := binlogdatapb.VReplicationWorkflowSubType_None - if mz.isPartial { - workflowSubType = binlogdatapb.VReplicationWorkflowSubType_Partial - } - var workflowType binlogdatapb.VReplicationWorkflowType - switch mz.ms.MaterializationIntent { - case vtctldatapb.MaterializationIntent_CUSTOM: - workflowType = binlogdatapb.VReplicationWorkflowType_Materialize - case vtctldatapb.MaterializationIntent_MOVETABLES: - workflowType = binlogdatapb.VReplicationWorkflowType_MoveTables - case vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX: - workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex + var workflowSubType binlogdatapb.VReplicationWorkflowSubType + workflowSubType, s, err := mz.getWorkflowSubType() + if err != nil { + return s, err } + workflowType := mz.getWorkflowType() + tabletTypeStr := mz.ms.TabletTypes if mz.ms.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { tabletTypeStr = discovery.InOrderHint + tabletTypeStr @@ -1414,6 +1409,34 @@ func (mz *materializer) generateInserts(ctx context.Context, targetShard *topo.S return ig.String(), nil } +func (mz *materializer) getWorkflowType() binlogdatapb.VReplicationWorkflowType { + var workflowType binlogdatapb.VReplicationWorkflowType + switch mz.ms.MaterializationIntent { + case vtctldatapb.MaterializationIntent_CUSTOM: + workflowType = binlogdatapb.VReplicationWorkflowType_Materialize + case vtctldatapb.MaterializationIntent_MOVETABLES: + workflowType = binlogdatapb.VReplicationWorkflowType_MoveTables + case vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX: + workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex + } + return workflowType +} + +func (mz *materializer) getWorkflowSubType() (binlogdatapb.VReplicationWorkflowSubType, string, error) { + workflowSubType := binlogdatapb.VReplicationWorkflowSubType_None + switch { + case mz.isPartial && mz.ms.AtomicCopy: + return workflowSubType, "", fmt.Errorf("both atomic copy and partial mode cannot be specified for the same workflow") + case mz.isPartial: + workflowSubType = binlogdatapb.VReplicationWorkflowSubType_Partial + case mz.ms.AtomicCopy: + workflowSubType = binlogdatapb.VReplicationWorkflowSubType_AtomicCopy + default: + workflowSubType = binlogdatapb.VReplicationWorkflowSubType_None + } + return workflowSubType, "", nil +} + func matchColInSelect(col sqlparser.IdentifierCI, sel *sqlparser.Select) (*sqlparser.ColName, error) { for _, selExpr := range sel.SelectExprs { switch selExpr := selExpr.(type) { diff --git a/go/vt/wrangler/vexec.go b/go/vt/wrangler/vexec.go index 40e4425c976..45a0e139c24 100644 --- a/go/vt/wrangler/vexec.go +++ b/go/vt/wrangler/vexec.go @@ -26,26 +26,25 @@ import ( "sync" "time" - "vitess.io/vitess/go/mysql/replication" - "vitess.io/vitess/go/textutil" - "vitess.io/vitess/go/vt/log" - "vitess.io/vitess/go/vt/topo/topoproto" - workflow2 "vitess.io/vitess/go/vt/vtctl/workflow" - "google.golang.org/protobuf/encoding/prototext" + "vitess.io/vitess/go/mysql/replication" "vitess.io/vitess/go/sets" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vterrors" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" + workflow2 "vitess.io/vitess/go/vt/vtctl/workflow" vtctldvexec "vitess.io/vitess/go/vt/vtctl/workflow/vexec" // renamed to avoid a collision with the vexec struct in this package ) @@ -360,6 +359,9 @@ func (wr *Wrangler) WorkflowAction(ctx context.Context, workflow, keyspace, acti default: } results, err := wr.execWorkflowAction(ctx, workflow, keyspace, action, dryRun, rpcReq) + if err != nil { + return nil, err + } if len(results) == 0 && !dryRun { // Dry runs produce no actual tablet results return nil, fmt.Errorf("the %s workflow does not exist in the %s keyspace", workflow, keyspace) } @@ -385,13 +387,42 @@ func (wr *Wrangler) getWorkflowActionQuery(action string) (string, error) { return query, nil } +// canStartWorkflow validates that, for an atomic copy workflow, none of the streams are still in the copy phase. +// Since we copy all tables in a single snapshot, we cannot restart a workflow which broke before all tables were copied. +func (wr *Wrangler) canStartWorkflow(ctx context.Context, workflow, keyspace string) error { + res, err := wr.ShowWorkflow(ctx, workflow, keyspace) + if err != nil { + return err + } + for _, shardStatus := range res.ShardStatuses { + if len(shardStatus.PrimaryReplicationStatuses) == 0 { + return fmt.Errorf("no replication streams found for workflow %s", workflow) + } + status := shardStatus.PrimaryReplicationStatuses[0] + + if status.WorkflowSubType == binlogdatapb.VReplicationWorkflowSubType_AtomicCopy.String() && + status.RowsCopied > 0 && len(status.CopyState) > 0 { + return fmt.Errorf("cannot restart an atomic copy workflow which previously stopped in the Copying phase") + } + break // We only need to check one shard + } + return nil +} + func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace, action string, dryRun bool, rpcReq any) (map[*topo.TabletInfo]*querypb.QueryResult, error) { var callback func(context.Context, *topo.TabletInfo) (*querypb.QueryResult, error) = nil query, err := wr.getWorkflowActionQuery(action) if err != nil { return nil, err } - if action == "update" { + + switch action { + case "start": + err = wr.canStartWorkflow(ctx, workflow, keyspace) + if err != nil { + return nil, err + } + case "update": rpcReq, ok := rpcReq.(*tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) if !ok { return nil, fmt.Errorf("invalid RPC request: %+v", rpcReq) @@ -443,6 +474,7 @@ func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace, } } } + return wr.runVexec(ctx, workflow, keyspace, query, callback, dryRun) } diff --git a/go/vt/wrangler/workflow.go b/go/vt/wrangler/workflow.go index 66c11f5c6ae..5670fb7173a 100644 --- a/go/vt/wrangler/workflow.go +++ b/go/vt/wrangler/workflow.go @@ -60,6 +60,7 @@ type VReplicationWorkflowParams struct { SourceTimeZone string DropForeignKeys bool InitializeTargetSequences bool + AtomicCopy bool // Reshard specific SourceShards, TargetShards []string @@ -436,7 +437,8 @@ func (vrw *VReplicationWorkflow) initMoveTables() error { return vrw.wr.MoveTables(vrw.ctx, vrw.params.Workflow, vrw.params.SourceKeyspace, vrw.params.TargetKeyspace, vrw.params.Tables, vrw.params.Cells, vrw.params.TabletTypes, vrw.params.AllTables, vrw.params.ExcludeTables, vrw.params.AutoStart, vrw.params.StopAfterCopy, vrw.params.ExternalCluster, vrw.params.DropForeignKeys, - vrw.params.DeferSecondaryKeys, vrw.params.SourceTimeZone, vrw.params.OnDDL, vrw.params.SourceShards, vrw.params.NoRoutingRules) + vrw.params.DeferSecondaryKeys, vrw.params.SourceTimeZone, vrw.params.OnDDL, vrw.params.SourceShards, + vrw.params.NoRoutingRules, vrw.params.AtomicCopy) } func (vrw *VReplicationWorkflow) initReshard() error { diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 2141aa99c3a..660f8fedfce 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -222,6 +222,7 @@ enum VReplicationWorkflowType { enum VReplicationWorkflowSubType { None = 0; Partial = 1; + AtomicCopy = 2; } // VReplicationWorklfowState defines the valid states that a workflow can be in. @@ -339,6 +340,7 @@ message RowEvent { repeated RowChange row_changes = 2; string keyspace = 3; string shard = 4; + uint32 flags = 5; // https://dev.mysql.com/doc/dev/mysql-server/latest/classbinary__log_1_1Rows__event.html } // FieldEvent represents the field info for a table. @@ -496,6 +498,24 @@ message VStreamRowsResponse { bool heartbeat = 7; } + +// VStreamTablesRequest is the payload for VStreamTables +message VStreamTablesRequest { + vtrpc.CallerID effective_caller_id = 1; + query.VTGateCallerID immediate_caller_id = 2; + query.Target target = 3; +} + +// VStreamTablesResponse is the response from VStreamTables +message VStreamTablesResponse { + string table_name = 1; + repeated query.Field fields = 2; + repeated query.Field pkfields = 3; + string gtid = 4; + repeated query.Row rows = 5; + query.Row lastpk = 6; +} + message LastPKEvent { TableLastPK table_last_p_k = 1; bool completed = 2; diff --git a/proto/queryservice.proto b/proto/queryservice.proto index eeed60aa50d..e2de692532c 100644 --- a/proto/queryservice.proto +++ b/proto/queryservice.proto @@ -106,6 +106,9 @@ service Query { // VStreamRows streams rows from the specified starting point. rpc VStreamRows(binlogdata.VStreamRowsRequest) returns (stream binlogdata.VStreamRowsResponse) {}; + // VStreamTables streams rows from the specified starting point. + rpc VStreamTables(binlogdata.VStreamTablesRequest) returns (stream binlogdata.VStreamTablesResponse) {}; + // VStreamResults streams results along with the gtid of the snapshot. rpc VStreamResults(binlogdata.VStreamResultsRequest) returns (stream binlogdata.VStreamResultsResponse) {}; diff --git a/proto/vtctldata.proto b/proto/vtctldata.proto index 8e3dd20f19c..a4828bed1d0 100644 --- a/proto/vtctldata.proto +++ b/proto/vtctldata.proto @@ -96,6 +96,7 @@ message MaterializeSettings { // DeferSecondaryKeys specifies if secondary keys should be created in one shot after table copy finishes. bool defer_secondary_keys = 14; tabletmanagerdata.TabletSelectionPreference tablet_selection_preference = 15; + bool atomic_copy = 16; } /* Data types for VtctldServer */ @@ -1023,6 +1024,8 @@ message MoveTablesCreateRequest { bool auto_start = 17; // NoRoutingRules is set to true if routing rules should not be created on the target when the workflow is created. bool no_routing_rules = 18; + // Run a single copy phase for the entire database. + bool atomic_copy = 19; } message MoveTablesCreateResponse { diff --git a/test/config.json b/test/config.json index b46152462c4..c28de9bb385 100644 --- a/test/config.json +++ b/test/config.json @@ -1166,6 +1166,15 @@ "RetryMax": 1, "Tags": [] }, + "vreplication_fk": { + "File": "unused.go", + "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestFKWorkflow"], + "Command": [], + "Manual": false, + "Shard": "vreplication_cellalias", + "RetryMax": 1, + "Tags": [] + }, "vreplication_across_db_versions": { "File": "unused.go", "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestV2WorkflowsAcrossDBVersions", "-timeout", "20m"], diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index d762cd44718..ef85a70d4ea 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -29722,7 +29722,8 @@ export namespace binlogdata { /** VReplicationWorkflowSubType enum. */ enum VReplicationWorkflowSubType { None = 0, - Partial = 1 + Partial = 1, + AtomicCopy = 2 } /** VReplicationWorkflowState enum. */ @@ -30153,6 +30154,9 @@ export namespace binlogdata { /** RowEvent shard */ shard?: (string|null); + + /** RowEvent flags */ + flags?: (number|null); } /** Represents a RowEvent. */ @@ -30176,6 +30180,9 @@ export namespace binlogdata { /** RowEvent shard. */ public shard: string; + /** RowEvent flags. */ + public flags: number; + /** * Creates a new RowEvent instance using the specified properties. * @param [properties] Properties to set @@ -31682,6 +31689,242 @@ export namespace binlogdata { public static getTypeUrl(typeUrlPrefix?: string): string; } + /** Properties of a VStreamTablesRequest. */ + interface IVStreamTablesRequest { + + /** VStreamTablesRequest effective_caller_id */ + effective_caller_id?: (vtrpc.ICallerID|null); + + /** VStreamTablesRequest immediate_caller_id */ + immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamTablesRequest target */ + target?: (query.ITarget|null); + } + + /** Represents a VStreamTablesRequest. */ + class VStreamTablesRequest implements IVStreamTablesRequest { + + /** + * Constructs a new VStreamTablesRequest. + * @param [properties] Properties to set + */ + constructor(properties?: binlogdata.IVStreamTablesRequest); + + /** VStreamTablesRequest effective_caller_id. */ + public effective_caller_id?: (vtrpc.ICallerID|null); + + /** VStreamTablesRequest immediate_caller_id. */ + public immediate_caller_id?: (query.IVTGateCallerID|null); + + /** VStreamTablesRequest target. */ + public target?: (query.ITarget|null); + + /** + * Creates a new VStreamTablesRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns VStreamTablesRequest instance + */ + public static create(properties?: binlogdata.IVStreamTablesRequest): binlogdata.VStreamTablesRequest; + + /** + * Encodes the specified VStreamTablesRequest message. Does not implicitly {@link binlogdata.VStreamTablesRequest.verify|verify} messages. + * @param message VStreamTablesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: binlogdata.IVStreamTablesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified VStreamTablesRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamTablesRequest.verify|verify} messages. + * @param message VStreamTablesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: binlogdata.IVStreamTablesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a VStreamTablesRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns VStreamTablesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamTablesRequest; + + /** + * Decodes a VStreamTablesRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns VStreamTablesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamTablesRequest; + + /** + * Verifies a VStreamTablesRequest message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a VStreamTablesRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns VStreamTablesRequest + */ + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamTablesRequest; + + /** + * Creates a plain object from a VStreamTablesRequest message. Also converts values to other types if specified. + * @param message VStreamTablesRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: binlogdata.VStreamTablesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this VStreamTablesRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for VStreamTablesRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a VStreamTablesResponse. */ + interface IVStreamTablesResponse { + + /** VStreamTablesResponse table_name */ + table_name?: (string|null); + + /** VStreamTablesResponse fields */ + fields?: (query.IField[]|null); + + /** VStreamTablesResponse pkfields */ + pkfields?: (query.IField[]|null); + + /** VStreamTablesResponse gtid */ + gtid?: (string|null); + + /** VStreamTablesResponse rows */ + rows?: (query.IRow[]|null); + + /** VStreamTablesResponse lastpk */ + lastpk?: (query.IRow|null); + } + + /** Represents a VStreamTablesResponse. */ + class VStreamTablesResponse implements IVStreamTablesResponse { + + /** + * Constructs a new VStreamTablesResponse. + * @param [properties] Properties to set + */ + constructor(properties?: binlogdata.IVStreamTablesResponse); + + /** VStreamTablesResponse table_name. */ + public table_name: string; + + /** VStreamTablesResponse fields. */ + public fields: query.IField[]; + + /** VStreamTablesResponse pkfields. */ + public pkfields: query.IField[]; + + /** VStreamTablesResponse gtid. */ + public gtid: string; + + /** VStreamTablesResponse rows. */ + public rows: query.IRow[]; + + /** VStreamTablesResponse lastpk. */ + public lastpk?: (query.IRow|null); + + /** + * Creates a new VStreamTablesResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns VStreamTablesResponse instance + */ + public static create(properties?: binlogdata.IVStreamTablesResponse): binlogdata.VStreamTablesResponse; + + /** + * Encodes the specified VStreamTablesResponse message. Does not implicitly {@link binlogdata.VStreamTablesResponse.verify|verify} messages. + * @param message VStreamTablesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: binlogdata.IVStreamTablesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified VStreamTablesResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamTablesResponse.verify|verify} messages. + * @param message VStreamTablesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: binlogdata.IVStreamTablesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a VStreamTablesResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns VStreamTablesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): binlogdata.VStreamTablesResponse; + + /** + * Decodes a VStreamTablesResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns VStreamTablesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): binlogdata.VStreamTablesResponse; + + /** + * Verifies a VStreamTablesResponse message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a VStreamTablesResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns VStreamTablesResponse + */ + public static fromObject(object: { [k: string]: any }): binlogdata.VStreamTablesResponse; + + /** + * Creates a plain object from a VStreamTablesResponse message. Also converts values to other types if specified. + * @param message VStreamTablesResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: binlogdata.VStreamTablesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this VStreamTablesResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for VStreamTablesResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of a LastPKEvent. */ interface ILastPKEvent { @@ -42123,6 +42366,9 @@ export namespace vtctldata { /** MaterializeSettings tablet_selection_preference */ tablet_selection_preference?: (tabletmanagerdata.TabletSelectionPreference|null); + + /** MaterializeSettings atomic_copy */ + atomic_copy?: (boolean|null); } /** Represents a MaterializeSettings. */ @@ -42179,6 +42425,9 @@ export namespace vtctldata { /** MaterializeSettings tablet_selection_preference. */ public tablet_selection_preference: tabletmanagerdata.TabletSelectionPreference; + /** MaterializeSettings atomic_copy. */ + public atomic_copy: boolean; + /** * Creates a new MaterializeSettings instance using the specified properties. * @param [properties] Properties to set @@ -54352,6 +54601,9 @@ export namespace vtctldata { /** MoveTablesCreateRequest no_routing_rules */ no_routing_rules?: (boolean|null); + + /** MoveTablesCreateRequest atomic_copy */ + atomic_copy?: (boolean|null); } /** Represents a MoveTablesCreateRequest. */ @@ -54417,6 +54669,9 @@ export namespace vtctldata { /** MoveTablesCreateRequest no_routing_rules. */ public no_routing_rules: boolean; + /** MoveTablesCreateRequest atomic_copy. */ + public atomic_copy: boolean; + /** * Creates a new MoveTablesCreateRequest instance using the specified properties. * @param [properties] Properties to set diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index d2d25fa11ab..c6c328a5cac 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -60639,6 +60639,7 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { return "workflow_sub_type: enum value expected"; case 0: case 1: + case 2: break; } if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) @@ -60808,6 +60809,10 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { case 1: message.workflow_sub_type = 1; break; + case "AtomicCopy": + case 2: + message.workflow_sub_type = 2; + break; } if (object.defer_secondary_keys != null) message.defer_secondary_keys = Boolean(object.defer_secondary_keys); @@ -62070,6 +62075,7 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { return "workflow_sub_type: enum value expected"; case 0: case 1: + case 2: break; } if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) @@ -62231,6 +62237,10 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { case 1: message.workflow_sub_type = 1; break; + case "AtomicCopy": + case 2: + message.workflow_sub_type = 2; + break; } if (object.defer_secondary_keys != null) message.defer_secondary_keys = Boolean(object.defer_secondary_keys); @@ -69279,11 +69289,13 @@ export const binlogdata = $root.binlogdata = (() => { * @enum {number} * @property {number} None=0 None value * @property {number} Partial=1 Partial value + * @property {number} AtomicCopy=2 AtomicCopy value */ binlogdata.VReplicationWorkflowSubType = (function() { const valuesById = {}, values = Object.create(valuesById); values[valuesById[0] = "None"] = 0; values[valuesById[1] = "Partial"] = 1; + values[valuesById[2] = "AtomicCopy"] = 2; return values; })(); @@ -70465,6 +70477,7 @@ export const binlogdata = $root.binlogdata = (() => { * @property {Array.|null} [row_changes] RowEvent row_changes * @property {string|null} [keyspace] RowEvent keyspace * @property {string|null} [shard] RowEvent shard + * @property {number|null} [flags] RowEvent flags */ /** @@ -70515,6 +70528,14 @@ export const binlogdata = $root.binlogdata = (() => { */ RowEvent.prototype.shard = ""; + /** + * RowEvent flags. + * @member {number} flags + * @memberof binlogdata.RowEvent + * @instance + */ + RowEvent.prototype.flags = 0; + /** * Creates a new RowEvent instance using the specified properties. * @function create @@ -70548,6 +70569,8 @@ export const binlogdata = $root.binlogdata = (() => { writer.uint32(/* id 3, wireType 2 =*/26).string(message.keyspace); if (message.shard != null && Object.hasOwnProperty.call(message, "shard")) writer.uint32(/* id 4, wireType 2 =*/34).string(message.shard); + if (message.flags != null && Object.hasOwnProperty.call(message, "flags")) + writer.uint32(/* id 5, wireType 0 =*/40).uint32(message.flags); return writer; }; @@ -70600,6 +70623,10 @@ export const binlogdata = $root.binlogdata = (() => { message.shard = reader.string(); break; } + case 5: { + message.flags = reader.uint32(); + break; + } default: reader.skipType(tag & 7); break; @@ -70653,6 +70680,9 @@ export const binlogdata = $root.binlogdata = (() => { if (message.shard != null && message.hasOwnProperty("shard")) if (!$util.isString(message.shard)) return "shard: string expected"; + if (message.flags != null && message.hasOwnProperty("flags")) + if (!$util.isInteger(message.flags)) + return "flags: integer expected"; return null; }; @@ -70684,6 +70714,8 @@ export const binlogdata = $root.binlogdata = (() => { message.keyspace = String(object.keyspace); if (object.shard != null) message.shard = String(object.shard); + if (object.flags != null) + message.flags = object.flags >>> 0; return message; }; @@ -70706,6 +70738,7 @@ export const binlogdata = $root.binlogdata = (() => { object.table_name = ""; object.keyspace = ""; object.shard = ""; + object.flags = 0; } if (message.table_name != null && message.hasOwnProperty("table_name")) object.table_name = message.table_name; @@ -70718,6 +70751,8 @@ export const binlogdata = $root.binlogdata = (() => { object.keyspace = message.keyspace; if (message.shard != null && message.hasOwnProperty("shard")) object.shard = message.shard; + if (message.flags != null && message.hasOwnProperty("flags")) + object.flags = message.flags; return object; }; @@ -74769,6 +74804,660 @@ export const binlogdata = $root.binlogdata = (() => { return VStreamRowsResponse; })(); + binlogdata.VStreamTablesRequest = (function() { + + /** + * Properties of a VStreamTablesRequest. + * @memberof binlogdata + * @interface IVStreamTablesRequest + * @property {vtrpc.ICallerID|null} [effective_caller_id] VStreamTablesRequest effective_caller_id + * @property {query.IVTGateCallerID|null} [immediate_caller_id] VStreamTablesRequest immediate_caller_id + * @property {query.ITarget|null} [target] VStreamTablesRequest target + */ + + /** + * Constructs a new VStreamTablesRequest. + * @memberof binlogdata + * @classdesc Represents a VStreamTablesRequest. + * @implements IVStreamTablesRequest + * @constructor + * @param {binlogdata.IVStreamTablesRequest=} [properties] Properties to set + */ + function VStreamTablesRequest(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * VStreamTablesRequest effective_caller_id. + * @member {vtrpc.ICallerID|null|undefined} effective_caller_id + * @memberof binlogdata.VStreamTablesRequest + * @instance + */ + VStreamTablesRequest.prototype.effective_caller_id = null; + + /** + * VStreamTablesRequest immediate_caller_id. + * @member {query.IVTGateCallerID|null|undefined} immediate_caller_id + * @memberof binlogdata.VStreamTablesRequest + * @instance + */ + VStreamTablesRequest.prototype.immediate_caller_id = null; + + /** + * VStreamTablesRequest target. + * @member {query.ITarget|null|undefined} target + * @memberof binlogdata.VStreamTablesRequest + * @instance + */ + VStreamTablesRequest.prototype.target = null; + + /** + * Creates a new VStreamTablesRequest instance using the specified properties. + * @function create + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {binlogdata.IVStreamTablesRequest=} [properties] Properties to set + * @returns {binlogdata.VStreamTablesRequest} VStreamTablesRequest instance + */ + VStreamTablesRequest.create = function create(properties) { + return new VStreamTablesRequest(properties); + }; + + /** + * Encodes the specified VStreamTablesRequest message. Does not implicitly {@link binlogdata.VStreamTablesRequest.verify|verify} messages. + * @function encode + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {binlogdata.IVStreamTablesRequest} message VStreamTablesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VStreamTablesRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.effective_caller_id != null && Object.hasOwnProperty.call(message, "effective_caller_id")) + $root.vtrpc.CallerID.encode(message.effective_caller_id, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.immediate_caller_id != null && Object.hasOwnProperty.call(message, "immediate_caller_id")) + $root.query.VTGateCallerID.encode(message.immediate_caller_id, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.target != null && Object.hasOwnProperty.call(message, "target")) + $root.query.Target.encode(message.target, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified VStreamTablesRequest message, length delimited. Does not implicitly {@link binlogdata.VStreamTablesRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {binlogdata.IVStreamTablesRequest} message VStreamTablesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VStreamTablesRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a VStreamTablesRequest message from the specified reader or buffer. + * @function decode + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {binlogdata.VStreamTablesRequest} VStreamTablesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VStreamTablesRequest.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamTablesRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.effective_caller_id = $root.vtrpc.CallerID.decode(reader, reader.uint32()); + break; + } + case 2: { + message.immediate_caller_id = $root.query.VTGateCallerID.decode(reader, reader.uint32()); + break; + } + case 3: { + message.target = $root.query.Target.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a VStreamTablesRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {binlogdata.VStreamTablesRequest} VStreamTablesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VStreamTablesRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a VStreamTablesRequest message. + * @function verify + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + VStreamTablesRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) { + let error = $root.vtrpc.CallerID.verify(message.effective_caller_id); + if (error) + return "effective_caller_id." + error; + } + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) { + let error = $root.query.VTGateCallerID.verify(message.immediate_caller_id); + if (error) + return "immediate_caller_id." + error; + } + if (message.target != null && message.hasOwnProperty("target")) { + let error = $root.query.Target.verify(message.target); + if (error) + return "target." + error; + } + return null; + }; + + /** + * Creates a VStreamTablesRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {Object.} object Plain object + * @returns {binlogdata.VStreamTablesRequest} VStreamTablesRequest + */ + VStreamTablesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamTablesRequest) + return object; + let message = new $root.binlogdata.VStreamTablesRequest(); + if (object.effective_caller_id != null) { + if (typeof object.effective_caller_id !== "object") + throw TypeError(".binlogdata.VStreamTablesRequest.effective_caller_id: object expected"); + message.effective_caller_id = $root.vtrpc.CallerID.fromObject(object.effective_caller_id); + } + if (object.immediate_caller_id != null) { + if (typeof object.immediate_caller_id !== "object") + throw TypeError(".binlogdata.VStreamTablesRequest.immediate_caller_id: object expected"); + message.immediate_caller_id = $root.query.VTGateCallerID.fromObject(object.immediate_caller_id); + } + if (object.target != null) { + if (typeof object.target !== "object") + throw TypeError(".binlogdata.VStreamTablesRequest.target: object expected"); + message.target = $root.query.Target.fromObject(object.target); + } + return message; + }; + + /** + * Creates a plain object from a VStreamTablesRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {binlogdata.VStreamTablesRequest} message VStreamTablesRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + VStreamTablesRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.effective_caller_id = null; + object.immediate_caller_id = null; + object.target = null; + } + if (message.effective_caller_id != null && message.hasOwnProperty("effective_caller_id")) + object.effective_caller_id = $root.vtrpc.CallerID.toObject(message.effective_caller_id, options); + if (message.immediate_caller_id != null && message.hasOwnProperty("immediate_caller_id")) + object.immediate_caller_id = $root.query.VTGateCallerID.toObject(message.immediate_caller_id, options); + if (message.target != null && message.hasOwnProperty("target")) + object.target = $root.query.Target.toObject(message.target, options); + return object; + }; + + /** + * Converts this VStreamTablesRequest to JSON. + * @function toJSON + * @memberof binlogdata.VStreamTablesRequest + * @instance + * @returns {Object.} JSON object + */ + VStreamTablesRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for VStreamTablesRequest + * @function getTypeUrl + * @memberof binlogdata.VStreamTablesRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + VStreamTablesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/binlogdata.VStreamTablesRequest"; + }; + + return VStreamTablesRequest; + })(); + + binlogdata.VStreamTablesResponse = (function() { + + /** + * Properties of a VStreamTablesResponse. + * @memberof binlogdata + * @interface IVStreamTablesResponse + * @property {string|null} [table_name] VStreamTablesResponse table_name + * @property {Array.|null} [fields] VStreamTablesResponse fields + * @property {Array.|null} [pkfields] VStreamTablesResponse pkfields + * @property {string|null} [gtid] VStreamTablesResponse gtid + * @property {Array.|null} [rows] VStreamTablesResponse rows + * @property {query.IRow|null} [lastpk] VStreamTablesResponse lastpk + */ + + /** + * Constructs a new VStreamTablesResponse. + * @memberof binlogdata + * @classdesc Represents a VStreamTablesResponse. + * @implements IVStreamTablesResponse + * @constructor + * @param {binlogdata.IVStreamTablesResponse=} [properties] Properties to set + */ + function VStreamTablesResponse(properties) { + this.fields = []; + this.pkfields = []; + this.rows = []; + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * VStreamTablesResponse table_name. + * @member {string} table_name + * @memberof binlogdata.VStreamTablesResponse + * @instance + */ + VStreamTablesResponse.prototype.table_name = ""; + + /** + * VStreamTablesResponse fields. + * @member {Array.} fields + * @memberof binlogdata.VStreamTablesResponse + * @instance + */ + VStreamTablesResponse.prototype.fields = $util.emptyArray; + + /** + * VStreamTablesResponse pkfields. + * @member {Array.} pkfields + * @memberof binlogdata.VStreamTablesResponse + * @instance + */ + VStreamTablesResponse.prototype.pkfields = $util.emptyArray; + + /** + * VStreamTablesResponse gtid. + * @member {string} gtid + * @memberof binlogdata.VStreamTablesResponse + * @instance + */ + VStreamTablesResponse.prototype.gtid = ""; + + /** + * VStreamTablesResponse rows. + * @member {Array.} rows + * @memberof binlogdata.VStreamTablesResponse + * @instance + */ + VStreamTablesResponse.prototype.rows = $util.emptyArray; + + /** + * VStreamTablesResponse lastpk. + * @member {query.IRow|null|undefined} lastpk + * @memberof binlogdata.VStreamTablesResponse + * @instance + */ + VStreamTablesResponse.prototype.lastpk = null; + + /** + * Creates a new VStreamTablesResponse instance using the specified properties. + * @function create + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {binlogdata.IVStreamTablesResponse=} [properties] Properties to set + * @returns {binlogdata.VStreamTablesResponse} VStreamTablesResponse instance + */ + VStreamTablesResponse.create = function create(properties) { + return new VStreamTablesResponse(properties); + }; + + /** + * Encodes the specified VStreamTablesResponse message. Does not implicitly {@link binlogdata.VStreamTablesResponse.verify|verify} messages. + * @function encode + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {binlogdata.IVStreamTablesResponse} message VStreamTablesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VStreamTablesResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.table_name != null && Object.hasOwnProperty.call(message, "table_name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.table_name); + if (message.fields != null && message.fields.length) + for (let i = 0; i < message.fields.length; ++i) + $root.query.Field.encode(message.fields[i], writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.pkfields != null && message.pkfields.length) + for (let i = 0; i < message.pkfields.length; ++i) + $root.query.Field.encode(message.pkfields[i], writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.gtid != null && Object.hasOwnProperty.call(message, "gtid")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.gtid); + if (message.rows != null && message.rows.length) + for (let i = 0; i < message.rows.length; ++i) + $root.query.Row.encode(message.rows[i], writer.uint32(/* id 5, wireType 2 =*/42).fork()).ldelim(); + if (message.lastpk != null && Object.hasOwnProperty.call(message, "lastpk")) + $root.query.Row.encode(message.lastpk, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified VStreamTablesResponse message, length delimited. Does not implicitly {@link binlogdata.VStreamTablesResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {binlogdata.IVStreamTablesResponse} message VStreamTablesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VStreamTablesResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a VStreamTablesResponse message from the specified reader or buffer. + * @function decode + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {binlogdata.VStreamTablesResponse} VStreamTablesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VStreamTablesResponse.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.binlogdata.VStreamTablesResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.table_name = reader.string(); + break; + } + case 2: { + if (!(message.fields && message.fields.length)) + message.fields = []; + message.fields.push($root.query.Field.decode(reader, reader.uint32())); + break; + } + case 3: { + if (!(message.pkfields && message.pkfields.length)) + message.pkfields = []; + message.pkfields.push($root.query.Field.decode(reader, reader.uint32())); + break; + } + case 4: { + message.gtid = reader.string(); + break; + } + case 5: { + if (!(message.rows && message.rows.length)) + message.rows = []; + message.rows.push($root.query.Row.decode(reader, reader.uint32())); + break; + } + case 6: { + message.lastpk = $root.query.Row.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a VStreamTablesResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {binlogdata.VStreamTablesResponse} VStreamTablesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VStreamTablesResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a VStreamTablesResponse message. + * @function verify + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + VStreamTablesResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.table_name != null && message.hasOwnProperty("table_name")) + if (!$util.isString(message.table_name)) + return "table_name: string expected"; + if (message.fields != null && message.hasOwnProperty("fields")) { + if (!Array.isArray(message.fields)) + return "fields: array expected"; + for (let i = 0; i < message.fields.length; ++i) { + let error = $root.query.Field.verify(message.fields[i]); + if (error) + return "fields." + error; + } + } + if (message.pkfields != null && message.hasOwnProperty("pkfields")) { + if (!Array.isArray(message.pkfields)) + return "pkfields: array expected"; + for (let i = 0; i < message.pkfields.length; ++i) { + let error = $root.query.Field.verify(message.pkfields[i]); + if (error) + return "pkfields." + error; + } + } + if (message.gtid != null && message.hasOwnProperty("gtid")) + if (!$util.isString(message.gtid)) + return "gtid: string expected"; + if (message.rows != null && message.hasOwnProperty("rows")) { + if (!Array.isArray(message.rows)) + return "rows: array expected"; + for (let i = 0; i < message.rows.length; ++i) { + let error = $root.query.Row.verify(message.rows[i]); + if (error) + return "rows." + error; + } + } + if (message.lastpk != null && message.hasOwnProperty("lastpk")) { + let error = $root.query.Row.verify(message.lastpk); + if (error) + return "lastpk." + error; + } + return null; + }; + + /** + * Creates a VStreamTablesResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {Object.} object Plain object + * @returns {binlogdata.VStreamTablesResponse} VStreamTablesResponse + */ + VStreamTablesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.binlogdata.VStreamTablesResponse) + return object; + let message = new $root.binlogdata.VStreamTablesResponse(); + if (object.table_name != null) + message.table_name = String(object.table_name); + if (object.fields) { + if (!Array.isArray(object.fields)) + throw TypeError(".binlogdata.VStreamTablesResponse.fields: array expected"); + message.fields = []; + for (let i = 0; i < object.fields.length; ++i) { + if (typeof object.fields[i] !== "object") + throw TypeError(".binlogdata.VStreamTablesResponse.fields: object expected"); + message.fields[i] = $root.query.Field.fromObject(object.fields[i]); + } + } + if (object.pkfields) { + if (!Array.isArray(object.pkfields)) + throw TypeError(".binlogdata.VStreamTablesResponse.pkfields: array expected"); + message.pkfields = []; + for (let i = 0; i < object.pkfields.length; ++i) { + if (typeof object.pkfields[i] !== "object") + throw TypeError(".binlogdata.VStreamTablesResponse.pkfields: object expected"); + message.pkfields[i] = $root.query.Field.fromObject(object.pkfields[i]); + } + } + if (object.gtid != null) + message.gtid = String(object.gtid); + if (object.rows) { + if (!Array.isArray(object.rows)) + throw TypeError(".binlogdata.VStreamTablesResponse.rows: array expected"); + message.rows = []; + for (let i = 0; i < object.rows.length; ++i) { + if (typeof object.rows[i] !== "object") + throw TypeError(".binlogdata.VStreamTablesResponse.rows: object expected"); + message.rows[i] = $root.query.Row.fromObject(object.rows[i]); + } + } + if (object.lastpk != null) { + if (typeof object.lastpk !== "object") + throw TypeError(".binlogdata.VStreamTablesResponse.lastpk: object expected"); + message.lastpk = $root.query.Row.fromObject(object.lastpk); + } + return message; + }; + + /** + * Creates a plain object from a VStreamTablesResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {binlogdata.VStreamTablesResponse} message VStreamTablesResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + VStreamTablesResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.fields = []; + object.pkfields = []; + object.rows = []; + } + if (options.defaults) { + object.table_name = ""; + object.gtid = ""; + object.lastpk = null; + } + if (message.table_name != null && message.hasOwnProperty("table_name")) + object.table_name = message.table_name; + if (message.fields && message.fields.length) { + object.fields = []; + for (let j = 0; j < message.fields.length; ++j) + object.fields[j] = $root.query.Field.toObject(message.fields[j], options); + } + if (message.pkfields && message.pkfields.length) { + object.pkfields = []; + for (let j = 0; j < message.pkfields.length; ++j) + object.pkfields[j] = $root.query.Field.toObject(message.pkfields[j], options); + } + if (message.gtid != null && message.hasOwnProperty("gtid")) + object.gtid = message.gtid; + if (message.rows && message.rows.length) { + object.rows = []; + for (let j = 0; j < message.rows.length; ++j) + object.rows[j] = $root.query.Row.toObject(message.rows[j], options); + } + if (message.lastpk != null && message.hasOwnProperty("lastpk")) + object.lastpk = $root.query.Row.toObject(message.lastpk, options); + return object; + }; + + /** + * Converts this VStreamTablesResponse to JSON. + * @function toJSON + * @memberof binlogdata.VStreamTablesResponse + * @instance + * @returns {Object.} JSON object + */ + VStreamTablesResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for VStreamTablesResponse + * @function getTypeUrl + * @memberof binlogdata.VStreamTablesResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + VStreamTablesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/binlogdata.VStreamTablesResponse"; + }; + + return VStreamTablesResponse; + })(); + binlogdata.LastPKEvent = (function() { /** @@ -102548,6 +103237,7 @@ export const vtctldata = $root.vtctldata = (() => { * @property {string|null} [on_ddl] MaterializeSettings on_ddl * @property {boolean|null} [defer_secondary_keys] MaterializeSettings defer_secondary_keys * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] MaterializeSettings tablet_selection_preference + * @property {boolean|null} [atomic_copy] MaterializeSettings atomic_copy */ /** @@ -102687,6 +103377,14 @@ export const vtctldata = $root.vtctldata = (() => { */ MaterializeSettings.prototype.tablet_selection_preference = 0; + /** + * MaterializeSettings atomic_copy. + * @member {boolean} atomic_copy + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.atomic_copy = false; + /** * Creates a new MaterializeSettings instance using the specified properties. * @function create @@ -102743,6 +103441,8 @@ export const vtctldata = $root.vtctldata = (() => { writer.uint32(/* id 14, wireType 0 =*/112).bool(message.defer_secondary_keys); if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) writer.uint32(/* id 15, wireType 0 =*/120).int32(message.tablet_selection_preference); + if (message.atomic_copy != null && Object.hasOwnProperty.call(message, "atomic_copy")) + writer.uint32(/* id 16, wireType 0 =*/128).bool(message.atomic_copy); return writer; }; @@ -102841,6 +103541,10 @@ export const vtctldata = $root.vtctldata = (() => { message.tablet_selection_preference = reader.int32(); break; } + case 16: { + message.atomic_copy = reader.bool(); + break; + } default: reader.skipType(tag & 7); break; @@ -102943,6 +103647,9 @@ export const vtctldata = $root.vtctldata = (() => { case 3: break; } + if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) + if (typeof message.atomic_copy !== "boolean") + return "atomic_copy: boolean expected"; return null; }; @@ -103037,6 +103744,8 @@ export const vtctldata = $root.vtctldata = (() => { message.tablet_selection_preference = 3; break; } + if (object.atomic_copy != null) + message.atomic_copy = Boolean(object.atomic_copy); return message; }; @@ -103071,6 +103780,7 @@ export const vtctldata = $root.vtctldata = (() => { object.on_ddl = ""; object.defer_secondary_keys = false; object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.atomic_copy = false; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; @@ -103108,6 +103818,8 @@ export const vtctldata = $root.vtctldata = (() => { object.defer_secondary_keys = message.defer_secondary_keys; if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) + object.atomic_copy = message.atomic_copy; return object; }; @@ -132298,6 +133010,7 @@ export const vtctldata = $root.vtctldata = (() => { * @property {boolean|null} [defer_secondary_keys] MoveTablesCreateRequest defer_secondary_keys * @property {boolean|null} [auto_start] MoveTablesCreateRequest auto_start * @property {boolean|null} [no_routing_rules] MoveTablesCreateRequest no_routing_rules + * @property {boolean|null} [atomic_copy] MoveTablesCreateRequest atomic_copy */ /** @@ -132464,6 +133177,14 @@ export const vtctldata = $root.vtctldata = (() => { */ MoveTablesCreateRequest.prototype.no_routing_rules = false; + /** + * MoveTablesCreateRequest atomic_copy. + * @member {boolean} atomic_copy + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.atomic_copy = false; + /** * Creates a new MoveTablesCreateRequest instance using the specified properties. * @function create @@ -132532,6 +133253,8 @@ export const vtctldata = $root.vtctldata = (() => { writer.uint32(/* id 17, wireType 0 =*/136).bool(message.auto_start); if (message.no_routing_rules != null && Object.hasOwnProperty.call(message, "no_routing_rules")) writer.uint32(/* id 18, wireType 0 =*/144).bool(message.no_routing_rules); + if (message.atomic_copy != null && Object.hasOwnProperty.call(message, "atomic_copy")) + writer.uint32(/* id 19, wireType 0 =*/152).bool(message.atomic_copy); return writer; }; @@ -132653,6 +133376,10 @@ export const vtctldata = $root.vtctldata = (() => { message.no_routing_rules = reader.bool(); break; } + case 19: { + message.atomic_copy = reader.bool(); + break; + } default: reader.skipType(tag & 7); break; @@ -132782,6 +133509,9 @@ export const vtctldata = $root.vtctldata = (() => { if (message.no_routing_rules != null && message.hasOwnProperty("no_routing_rules")) if (typeof message.no_routing_rules !== "boolean") return "no_routing_rules: boolean expected"; + if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) + if (typeof message.atomic_copy !== "boolean") + return "atomic_copy: boolean expected"; return null; }; @@ -132926,6 +133656,8 @@ export const vtctldata = $root.vtctldata = (() => { message.auto_start = Boolean(object.auto_start); if (object.no_routing_rules != null) message.no_routing_rules = Boolean(object.no_routing_rules); + if (object.atomic_copy != null) + message.atomic_copy = Boolean(object.atomic_copy); return message; }; @@ -132963,6 +133695,7 @@ export const vtctldata = $root.vtctldata = (() => { object.defer_secondary_keys = false; object.auto_start = false; object.no_routing_rules = false; + object.atomic_copy = false; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; @@ -133015,6 +133748,8 @@ export const vtctldata = $root.vtctldata = (() => { object.auto_start = message.auto_start; if (message.no_routing_rules != null && message.hasOwnProperty("no_routing_rules")) object.no_routing_rules = message.no_routing_rules; + if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) + object.atomic_copy = message.atomic_copy; return object; }; From 543ab3b99a7b4f6a3e94479b85698df9eac2e22b Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Tue, 5 Sep 2023 22:17:03 +0200 Subject: [PATCH 2/7] Address Shlomi's review comments Signed-off-by: Rohit Nayak --- go/cmd/vtctldclient/command/workflows.go | 6 +++++ go/test/endtoend/vreplication/fk_test.go | 22 ++++++++----------- .../tabletmanager/vreplication/vcopier.go | 9 ++++++++ ...{vcopier_copy_all.go => vcopier_atomic.go} | 20 ++++++++++++----- .../tabletserver/vstreamer/snapshot_conn.go | 7 +++++- .../tabletserver/vstreamer/tablestreamer.go | 15 ++++++++++++- 6 files changed, 59 insertions(+), 20 deletions(-) rename go/vt/vttablet/tabletmanager/vreplication/{vcopier_copy_all.go => vcopier_atomic.go} (93%) diff --git a/go/cmd/vtctldclient/command/workflows.go b/go/cmd/vtctldclient/command/workflows.go index a9ef6df88c7..6d8d53ae6f4 100644 --- a/go/cmd/vtctldclient/command/workflows.go +++ b/go/cmd/vtctldclient/command/workflows.go @@ -21,6 +21,9 @@ import ( "sort" "strings" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vterrors" + "github.com/spf13/cobra" "vitess.io/vitess/go/cmd/vtctldclient/cli" @@ -244,6 +247,9 @@ func canStartWorkflow(keyspace, workflow string) error { if len(resp.Workflows) == 0 { return fmt.Errorf("workflow %s not found", workflow) } + if len(resp.Workflows) > 1 { + return vterrors.Errorf(vtrpc.Code_INTERNAL, "multiple results found for workflow %s", workflow) + } wf := resp.Workflows[0] if wf.WorkflowSubType != binlogdatapb.VReplicationWorkflowSubType_AtomicCopy.String() { return nil diff --git a/go/test/endtoend/vreplication/fk_test.go b/go/test/endtoend/vreplication/fk_test.go index f96e0ddd8ac..69a8d316a52 100644 --- a/go/test/endtoend/vreplication/fk_test.go +++ b/go/test/endtoend/vreplication/fk_test.go @@ -38,9 +38,14 @@ import ( // It inserts initial data, then simulates load. We insert both child rows with foreign keys and those without, // i.e. with foreign_key_checks=0. func TestFKWorkflow(t *testing.T) { + // ensure that there are multiple copy phase cycles per table + extraVTTabletArgs = []string{"--vstream_packet_size=256"} + defer func() { extraVTTabletArgs = nil }() + cellName := "zone" cells := []string{cellName} vc = NewVitessCluster(t, "TestFKWorkflow", cells, mainClusterConfig) + require.NotNil(t, vc) allCellNames = cellName defaultCellName := cellName @@ -103,9 +108,7 @@ func TestFKWorkflow(t *testing.T) { waitForAdditionalRows(t, 200) vdiff(t, targetKeyspace, workflowName, cellName, true, false, nil) if withLoad { - go func() { - cancel() - }() + cancel() <-ch } mt.SwitchReadsAndWrites() @@ -114,20 +117,12 @@ func TestFKWorkflow(t *testing.T) { if withLoad { ctx, cancel = context.WithCancel(context.Background()) - defer func() { - select { - case <-ctx.Done(): - default: - cancel() - } - }() + defer cancel() go simulateLoad(t, ctx) } waitForAdditionalRows(t, 200) if withLoad { - go func() { - cancel() - }() + cancel() <-ch } } @@ -221,6 +216,7 @@ func insert(t *testing.T) { insertQuery := fmt.Sprintf("INSERT INTO parent (id) VALUES (%d)", currentParentId) qr := exec2(t, insertQuery) require.NotNil(t, qr) + // insert one or more children, some with valid foreign keys, some without. for i := 0; i < rand.Intn(4)+1; i++ { currentChildId++ if i == 3 { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go index cb9c7100522..f3303374169 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -509,6 +509,12 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma rows = rows.CloneVT() } + // Code below is copied from vcopier.go. It was implemented to facilitate + // parallel bulk inserts in https://github.com/vitessio/vitess/pull/10828. + // We can probably extract this into a common package and use it for both + // flavors of the vcopier. But cut/pasting it for now, so as to not change + // vcopier at the moment to avoid any regressions. + // Prepare a vcopierCopyTask for the current batch of work. // TODO(maxeng) see if using a pre-allocated pool will speed things up. currCh := make(chan *vcopierCopyTaskResult, 1) @@ -671,6 +677,8 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma return nil } +// updatePos is called after the last table is copied in an atomic copy, to set the gtid so that the replicating phase +// can start from the gtid where the snapshot with all tables was taken. It also updates the final copy row count. func (vc *vcopier) updatePos(ctx context.Context, gtid string) error { pos, err := replication.DecodePosition(gtid) if err != nil { @@ -1215,6 +1223,7 @@ func vcopierCopyTaskGetNextState(vts vcopierCopyTaskState) vcopierCopyTaskState return vts } +// getInsertParallelism returns the number of parallel workers to use for inserting batches during the copy phase. func getInsertParallelism() int { parallelism := int(math.Max(1, float64(vreplicationParallelInsertWorkers))) return parallelism diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go similarity index 93% rename from go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go rename to go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go index 7a9df98d000..98b46c3e2a0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_copy_all.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go @@ -82,15 +82,15 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings rowsCopiedTicker := time.NewTicker(rowsCopiedUpdateInterval) defer rowsCopiedTicker.Stop() - copyStateGCTicker := time.NewTicker(copyStateGCInterval) - defer copyStateGCTicker.Stop() parallelism := getInsertParallelism() copyWorkerFactory := vc.newCopyWorkerFactory(parallelism) var copyWorkQueue *vcopierCopyWorkQueue - // Allocate a result channel to collect results from tasks. - resultCh := make(chan *vcopierCopyTaskResult, parallelism*4) + // Allocate a result channel to collect results from tasks. To not block fast workers, we allocate a buffer of + // MaxResultsInFlight results per worker. + const MaxResultsInFlight = 4 + resultCh := make(chan *vcopierCopyTaskResult, parallelism*MaxResultsInFlight) defer close(resultCh) var lastpk *querypb.Row @@ -142,10 +142,18 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings state.currentTableName = tableName } + + // A new copy queue is created for each table. The queue is closed when the table is done. if !copyWorkQueue.isOpen { if len(resp.Fields) == 0 { return fmt.Errorf("expecting field event first, got: %v", resp) } + + lastpk = nil + // pkfields are only used for logging, so that we can monitor progress. + pkfields = make([]*querypb.Field, len(resp.Pkfields)) + copy(pkfields, resp.Pkfields) + fieldEvent := &binlogdatapb.FieldEvent{ TableName: tableName, } @@ -154,7 +162,7 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings if err != nil { return err } - pkfields = append(pkfields, pkfields...) + buf := sqlparser.NewTrackedBuffer(nil) buf.Myprintf( "insert into _vt.copy_state (lastpk, vrepl_id, table_name) values (%a, %s, %s)", ":lastpk", @@ -163,6 +171,7 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings addLatestCopyState := buf.ParsedQuery() copyWorkQueue.open(addLatestCopyState, pkfields, tablePlan) } + // When rowstreamer has finished streaming all rows, we get a callback with empty rows. if len(resp.Rows) == 0 { return nil } @@ -171,6 +180,7 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings Fields: pkfields, Rows: []*querypb.Row{lastpk}, }) + if merr != nil { return fmt.Errorf("failed to marshal pk fields and value into query result: %s", merr.Error()) } diff --git a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go index c3fead67508..b9a3a70ea98 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go +++ b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "sync/atomic" + "time" "github.com/spf13/pflag" @@ -221,7 +222,11 @@ func SetBinlogRotationThreshold(threshold int64) { // startSnapshotAllTables starts a streaming query with a snapshot view of all tables, returning the // GTID set from the time when the snapshot was taken. func (conn *snapshotConn) startSnapshotAllTables(ctx context.Context) (gtid string, err error) { - lockConn, err := mysqlConnect(ctx, conn.cp) + const MaxLockWaitTime = 30 * time.Second + shortCtx, cancel := context.WithTimeout(ctx, MaxLockWaitTime) + defer cancel() + + lockConn, err := mysqlConnect(shortCtx, conn.cp) if err != nil { return "", err } diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go index 31f9d23de87..7614e1a9924 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go @@ -29,6 +29,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + schema2 "vitess.io/vitess/go/vt/schema" ) /* @@ -90,7 +91,15 @@ func (ts *tableStreamer) Stream() error { defer conn.Close() ts.snapshotConn = conn - if ts.gtid, err = conn.startSnapshotAllTables(ts.ctx); err != nil { + _, err = conn.ExecuteFetch("set session session_track_gtids = START_GTID", 1, false) + if err != nil { + // session_track_gtids = START_GTID unsupported or cannot execute. Resort to LOCK-based snapshot + ts.gtid, err = conn.startSnapshotAllTables(ts.ctx) + } else { + // session_track_gtids = START_GTID supported. Get a transaction with consistent GTID without LOCKing tables. + ts.gtid, err = conn.startSnapshotWithConsistentGTID(ts.ctx) + } + if err != nil { return err } @@ -104,6 +113,10 @@ func (ts *tableStreamer) Stream() error { } for _, row := range rs.Rows { tableName := row[0].ToString() + if schema2.IsInternalOperationTableName(tableName) { + log.Infof("Skipping internal table %s", tableName) + continue + } ts.tables = append(ts.tables, tableName) } log.Infof("Found %d tables to stream: %+v", len(ts.tables), ts.tables) From a7a70e3ef79ec66f2688149f67ef2e8949f0f193 Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Tue, 5 Sep 2023 22:31:49 +0200 Subject: [PATCH 3/7] Fix merge issue Signed-off-by: Rohit Nayak --- go/vt/wrangler/materializer_test.go | 18 +++++++++--------- go/vt/wrangler/traffic_switcher_test.go | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/go/vt/wrangler/materializer_test.go b/go/vt/wrangler/materializer_test.go index 9296c6f70d4..089be4c10ea 100644 --- a/go/vt/wrangler/materializer_test.go +++ b/go/vt/wrangler/materializer_test.go @@ -71,7 +71,7 @@ func TestMoveTablesNoRoutingRules(t *testing.T) { env.tmc.expectVRQuery(200, mzSelectIDQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, true) + err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, true, false) require.NoError(t, err) rr, err := env.wr.ts.GetRoutingRules(ctx) require.NoError(t, err) @@ -99,7 +99,7 @@ func TestMigrateTables(t *testing.T) { env.tmc.expectVRQuery(200, mzSelectIDQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false) + err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false, false) require.NoError(t, err) vschema, err := env.wr.ts.GetSrvVSchema(ctx, env.cell) require.NoError(t, err) @@ -142,11 +142,11 @@ func TestMissingTables(t *testing.T) { env.tmc.expectVRQuery(200, mzSelectIDQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1,tyt", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false) + err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1,tyt", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false, false) require.EqualError(t, err, "table(s) not found in source keyspace sourceks: tyt") - err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1,tyt,t2,txt", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false) + err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1,tyt,t2,txt", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false, false) require.EqualError(t, err, "table(s) not found in source keyspace sourceks: tyt,txt") - err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false) + err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false, false) require.NoError(t, err) } @@ -204,7 +204,7 @@ func TestMoveTablesAllAndExclude(t *testing.T) { env.tmc.expectVRQuery(200, insertPrefix, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzSelectIDQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "", "", "", tcase.allTables, tcase.excludeTables, true, false, "", false, false, "", defaultOnDDL, nil, false) + err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "", "", "", tcase.allTables, tcase.excludeTables, true, false, "", false, false, "", defaultOnDDL, nil, false, false) require.NoError(t, err) require.EqualValues(t, tcase.want, targetTables(ctx, env)) }) @@ -240,7 +240,7 @@ func TestMoveTablesStopFlags(t *testing.T) { env.tmc.expectVRQuery(200, mzSelectIDQuery, &sqltypes.Result{}) // -auto_start=false is tested by NOT expecting the update query which sets state to RUNNING err = env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", - "", false, "", false, true, "", false, false, "", defaultOnDDL, nil, false) + "", false, "", false, true, "", false, false, "", defaultOnDDL, nil, false, false) require.NoError(t, err) env.tmc.verifyQueries(t) }) @@ -268,7 +268,7 @@ func TestMigrateVSchema(t *testing.T) { env.tmc.expectVRQuery(200, mzSelectIDQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", `{"t1":{}}`, "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false) + err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", `{"t1":{}}`, "", "", false, "", true, false, "", false, false, "", defaultOnDDL, nil, false, false) require.NoError(t, err) vschema, err := env.wr.ts.GetSrvVSchema(ctx, env.cell) require.NoError(t, err) @@ -3092,7 +3092,7 @@ func TestMoveTablesDDLFlag(t *testing.T) { env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) err := env.wr.MoveTables(ctx, "workflow", "sourceks", "targetks", "t1", "", - "", false, "", false, true, "", false, false, "", onDDLAction, nil, false) + "", false, "", false, true, "", false, false, "", onDDLAction, nil, false, false) require.NoError(t, err) }) } diff --git a/go/vt/wrangler/traffic_switcher_test.go b/go/vt/wrangler/traffic_switcher_test.go index c0fc5fd82ed..91bfe8a6445 100644 --- a/go/vt/wrangler/traffic_switcher_test.go +++ b/go/vt/wrangler/traffic_switcher_test.go @@ -2169,7 +2169,7 @@ func TestNoOrphanedRoutingRulesOnFailedCreate(t *testing.T) { err = topotools.RebuildKeyspace(ctx, logutil.NewConsoleLogger(), tme.ts, "ks1", []string{"cell1"}, false) require.NoError(t, err, "failed to rebuild keyspace") - err = tme.wr.MoveTables(ctx, "testwf", "ks1", "ks2", "t1,t2", "cell1", "primary,replica", false, "", true, false, "", false, false, "", "", nil, false) + err = tme.wr.MoveTables(ctx, "testwf", "ks1", "ks2", "t1,t2", "cell1", "primary,replica", false, "", true, false, "", false, false, "", "", nil, false, false) require.Error(t, err) // Check that there are no orphaned routing rules. From 55b621541b19ba724597f8ab116f0669e643cacc Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Wed, 6 Sep 2023 11:16:48 +0200 Subject: [PATCH 4/7] Address some review comments Signed-off-by: Rohit Nayak --- go/cmd/vtctldclient/command/movetables.go | 35 ++++---- go/test/endtoend/vreplication/fk_test.go | 84 ++++++++++++------- .../tabletserver/vstreamer/tablestreamer.go | 5 +- .../vstreamer/tablestreamer_test.go | 2 +- 4 files changed, 74 insertions(+), 52 deletions(-) diff --git a/go/cmd/vtctldclient/command/movetables.go b/go/cmd/vtctldclient/command/movetables.go index 964e684ee44..9115578f411 100644 --- a/go/cmd/vtctldclient/command/movetables.go +++ b/go/cmd/vtctldclient/command/movetables.go @@ -79,24 +79,6 @@ See the --help output for each command for more details.`, RunE: commandMoveTablesComplete, } - checkAtomicCopyOptions = func() error { - var errors []string - if !moveTablesCreateOptions.AtomicCopy { - return nil - } - if !moveTablesCreateOptions.AllTables { - errors = append(errors, "atomic copy requires --all-tables.") - } - if len(moveTablesCreateOptions.IncludeTables) > 0 || len(moveTablesCreateOptions.ExcludeTables) > 0 { - errors = append(errors, "atomic copy does not support specifying tables.") - } - if len(errors) > 0 { - errors = append(errors, "Found options incompatible with atomic copy:") - return fmt.Errorf(strings.Join(errors, " ")) - } - return nil - } - // MoveTablesCreate makes a MoveTablesCreate gRPC call to a vtctld. MoveTablesCreate = &cobra.Command{ Use: "create", @@ -124,6 +106,23 @@ See the --help output for each command for more details.`, return fmt.Errorf("invalid on-ddl value: %s", moveTablesCreateOptions.OnDDL) } + checkAtomicCopyOptions := func() error { + var errors []string + if !moveTablesCreateOptions.AtomicCopy { + return nil + } + if !moveTablesCreateOptions.AllTables { + errors = append(errors, "atomic copy requires --all-tables.") + } + if len(moveTablesCreateOptions.IncludeTables) > 0 || len(moveTablesCreateOptions.ExcludeTables) > 0 { + errors = append(errors, "atomic copy does not support specifying tables.") + } + if len(errors) > 0 { + errors = append(errors, "Found options incompatible with atomic copy:") + return fmt.Errorf(strings.Join(errors, " ")) + } + return nil + } if err := checkAtomicCopyOptions(); err != nil { return err } diff --git a/go/test/endtoend/vreplication/fk_test.go b/go/test/endtoend/vreplication/fk_test.go index 69a8d316a52..31886864f11 100644 --- a/go/test/endtoend/vreplication/fk_test.go +++ b/go/test/endtoend/vreplication/fk_test.go @@ -68,12 +68,15 @@ func TestFKWorkflow(t *testing.T) { defer vtgateConn.Close() verifyClusterHealth(t, vc) + var ls *fkLoadSimulator + insertInitialFKData(t) withLoad := true // Set it to false to skip load simulation, while debugging var cancel context.CancelFunc var ctx context.Context if withLoad { ctx, cancel = context.WithCancel(context.Background()) + ls = newFKLoadSimulator(t, ctx) defer func() { select { case <-ctx.Done(): @@ -81,7 +84,7 @@ func TestFKWorkflow(t *testing.T) { cancel() } }() - go simulateLoad(t, ctx) + go ls.simulateLoad() } targetKeyspace := "fktarget" targetTabletId := 200 @@ -105,7 +108,7 @@ func TestFKWorkflow(t *testing.T) { require.NotNil(t, targetTab) catchup(t, targetTab, workflowName, "MoveTables") vdiff(t, targetKeyspace, workflowName, cellName, true, false, nil) - waitForAdditionalRows(t, 200) + ls.waitForAdditionalRows(200) vdiff(t, targetKeyspace, workflowName, cellName, true, false, nil) if withLoad { cancel() @@ -117,10 +120,11 @@ func TestFKWorkflow(t *testing.T) { if withLoad { ctx, cancel = context.WithCancel(context.Background()) + ls = newFKLoadSimulator(t, ctx) defer cancel() - go simulateLoad(t, ctx) + go ls.simulateLoad() } - waitForAdditionalRows(t, 200) + ls.waitForAdditionalRows(200) if withLoad { cancel() <-ch @@ -150,14 +154,27 @@ func init() { var ch = make(chan bool) -func simulateLoad(t *testing.T, ctx context.Context) { +type fkLoadSimulator struct { + t *testing.T + ctx context.Context +} + +func newFKLoadSimulator(t *testing.T, ctx context.Context) *fkLoadSimulator { + return &fkLoadSimulator{ + t: t, + ctx: ctx, + } +} + +func (ls *fkLoadSimulator) simulateLoad() { + t := ls.t var err error for i := 0; ; i++ { if i%1000 == 0 { log.Infof("Load simulation iteration %d", i) } select { - case <-ctx.Done(): + case <-ls.ctx.Done(): ch <- true return default: @@ -166,18 +183,19 @@ func simulateLoad(t *testing.T, ctx context.Context) { op := rand.Intn(100) switch { case op < 50: // 50% chance to insert - insert(t) + ls.insert() case op < 80: // 30% chance to update - update(t) + ls.update() default: // 20% chance to delete - delete(t) + ls.delete() } require.NoError(t, err) time.Sleep(1 * time.Millisecond) } } -func getNumRowsParent(t *testing.T, vtgateConn *mysql.Conn) int { +func (ls *fkLoadSimulator) getNumRowsParent(vtgateConn *mysql.Conn) int { + t := ls.t qr := execVtgateQuery(t, vtgateConn, "fksource", "SELECT COUNT(*) FROM parent") require.NotNil(t, qr) numRows, err := strconv.Atoi(qr.Rows[0][0].ToString()) @@ -185,10 +203,11 @@ func getNumRowsParent(t *testing.T, vtgateConn *mysql.Conn) int { return numRows } -func waitForAdditionalRows(t *testing.T, count int) { +func (ls *fkLoadSimulator) waitForAdditionalRows(count int) { + t := ls.t vtgateConn := getConnection(t, vc.ClusterConfig.hostname, vc.ClusterConfig.vtgateMySQLPort) defer vtgateConn.Close() - numRowsStart := getNumRowsParent(t, vtgateConn) + numRowsStart := ls.getNumRowsParent(vtgateConn) shortCtx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() for { @@ -196,7 +215,7 @@ func waitForAdditionalRows(t *testing.T, count int) { case shortCtx.Err() != nil: t.Fatalf("Timed out waiting for additional rows") default: - numRows := getNumRowsParent(t, vtgateConn) + numRows := ls.getNumRowsParent(vtgateConn) if numRows >= numRowsStart+count { return } @@ -205,33 +224,29 @@ func waitForAdditionalRows(t *testing.T, count int) { } } -func exec2(t *testing.T, query string) *sqltypes.Result { - qr := execVtgateQuery(t, vtgateConn, "fksource", query) - require.NotNil(t, qr) - return qr -} - -func insert(t *testing.T) { +func (ls *fkLoadSimulator) insert() { + t := ls.t currentParentId++ insertQuery := fmt.Sprintf("INSERT INTO parent (id) VALUES (%d)", currentParentId) - qr := exec2(t, insertQuery) + qr := ls.exec(insertQuery) require.NotNil(t, qr) // insert one or more children, some with valid foreign keys, some without. for i := 0; i < rand.Intn(4)+1; i++ { currentChildId++ if i == 3 { insertQuery = fmt.Sprintf("INSERT /*+ SET_VAR(foreign_key_checks=0) */ INTO child (id, parent_id) VALUES (%d, %d)", currentChildId, currentParentId+1000000) - exec2(t, insertQuery) + ls.exec(insertQuery) } else { insertQuery = fmt.Sprintf("INSERT INTO child (id, parent_id) VALUES (%d, %d)", currentChildId, currentParentId) - exec2(t, insertQuery) + ls.exec(insertQuery) } } } -func getRandomId(t *testing.T) int64 { +func (ls *fkLoadSimulator) getRandomId() int64 { + t := ls.t selectQuery := "SELECT id FROM parent ORDER BY RAND() LIMIT 1" - qr := exec2(t, selectQuery) + qr := ls.exec(selectQuery) require.NotNil(t, qr) if len(qr.Rows) == 0 { return 0 @@ -241,12 +256,19 @@ func getRandomId(t *testing.T) int64 { return id } -func update(t *testing.T) { - updateQuery := fmt.Sprintf("UPDATE parent SET name = 'parent%d' WHERE id = %d", rand.Intn(1000)+1, getRandomId(t)) - exec2(t, updateQuery) +func (ls *fkLoadSimulator) update() { + updateQuery := fmt.Sprintf("UPDATE parent SET name = 'parent%d' WHERE id = %d", rand.Intn(1000)+1, ls.getRandomId()) + ls.exec(updateQuery) +} + +func (ls *fkLoadSimulator) delete() { + deleteQuery := fmt.Sprintf("DELETE FROM parent WHERE id = %d", ls.getRandomId()) + ls.exec(deleteQuery) } -func delete(t *testing.T) { - deleteQuery := fmt.Sprintf("DELETE FROM parent WHERE id = %d", getRandomId(t)) - exec2(t, deleteQuery) +func (ls *fkLoadSimulator) exec(query string) *sqltypes.Result { + t := ls.t + qr := execVtgateQuery(t, vtgateConn, "fksource", query) + require.NotNil(t, qr) + return qr } diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go index 7614e1a9924..417e8a5c993 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go @@ -59,7 +59,8 @@ type tableStreamer struct { gtid string } -func newTableStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, vschema *localVSchema, send func(response *binlogdatapb.VStreamTablesResponse) error, vse *Engine) *tableStreamer { +func newTableStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, vschema *localVSchema, + send func(response *binlogdatapb.VStreamTablesResponse) error, vse *Engine) *tableStreamer { ctx, cancel := context.WithCancel(ctx) return &tableStreamer{ ctx: ctx, @@ -78,7 +79,7 @@ func (ts *tableStreamer) Cancel() { } func (ts *tableStreamer) Stream() error { - // Ensure that the schema engine is Open. If vttablet came up in a non_serving role, it may not have been initialized. + // Ensure that the schema engine is Open. If vttablet came up non_serving, it may not have been initialized. var err error if err = ts.se.Open(); err != nil { return err diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go index 1062e5e4a03..bc6ba98d636 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go @@ -26,7 +26,7 @@ import ( binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) -// TestTableStreamer streams multiple tables and ensures all rows are received in the correct order. +// TestTableStreamer streams all tables and ensures all rows are received in the correct order. func TestTableStreamer(t *testing.T) { ctx := context.Background() execStatements(t, []string{ From c6faa44ab587e9a414d95a3c68d5ddb3d833da01 Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Wed, 6 Sep 2023 11:32:22 +0200 Subject: [PATCH 5/7] Address more review comments Signed-off-by: Rohit Nayak --- go/cmd/vtctldclient/command/workflows.go | 6 +++--- go/vt/vttablet/tabletmanager/vreplication/vcopier.go | 2 +- go/vt/vttablet/tabletmanager/vreplication/vreplicator.go | 2 +- go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go | 3 ++- go/vt/wrangler/vexec.go | 6 +++--- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/go/cmd/vtctldclient/command/workflows.go b/go/cmd/vtctldclient/command/workflows.go index 6d8d53ae6f4..902a8a92ce6 100644 --- a/go/cmd/vtctldclient/command/workflows.go +++ b/go/cmd/vtctldclient/command/workflows.go @@ -237,9 +237,9 @@ func getWorkflow(keyspace, workflow string) (*vtctldatapb.GetWorkflowsResponse, return resp, nil } -// canStartWorkflow validates that, for an atomic copy workflow, none of the streams are still in the copy phase. +// canRestartWorkflow validates that, for an atomic copy workflow, none of the streams are still in the copy phase. // Since we copy all tables in a single snapshot, we cannot restart a workflow which broke before all tables were copied. -func canStartWorkflow(keyspace, workflow string) error { +func canRestartWorkflow(keyspace, workflow string) error { resp, err := getWorkflow(keyspace, workflow) if err != nil { return err @@ -354,7 +354,7 @@ func commandWorkflowUpdateState(cmd *cobra.Command, args []string) error { var state binlogdatapb.VReplicationWorkflowState switch strings.ToLower(cmd.Name()) { case "start": - if err := canStartWorkflow(workflowUpdateOptions.Workflow, workflowOptions.Keyspace); err != nil { + if err := canRestartWorkflow(workflowUpdateOptions.Workflow, workflowOptions.Keyspace); err != nil { return err } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go index f3303374169..ebfe0e22343 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -1092,7 +1092,7 @@ func (vbc *vcopierCopyWorker) execute(ctx context.Context, task *vcopierCopyTask } case vcopierCopyTaskInsertCopyState: advanceFn = func(ctx context.Context, args *vcopierCopyTaskArgs) error { - if vbc.copyStateInsert == nil { + if vbc.copyStateInsert == nil { // we don't insert copy state for atomic copy log.Infof("Skipping copy_state insert") return nil } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index c8d04c80294..ca2d84bd3c4 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -274,7 +274,7 @@ func (vr *vreplicator) replicate(ctx context.Context) error { } if vr.WorkflowSubType == int32(binlogdatapb.VReplicationWorkflowSubType_AtomicCopy) { if err := newVCopier(vr).copyAll(ctx, settings); err != nil { - log.Infof("Error copying all tables: %v", err) + log.Infof("Error atomically copying all tables: %v", err) vr.stats.ErrorCounts.Add([]string{"CopyAll"}, 1) return err } diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go index 417e8a5c993..1039f21b4d6 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go @@ -20,6 +20,7 @@ import ( "context" "errors" "fmt" + "strings" "sync/atomic" "vitess.io/vitess/go/sqlescape" @@ -120,7 +121,7 @@ func (ts *tableStreamer) Stream() error { } ts.tables = append(ts.tables, tableName) } - log.Infof("Found %d tables to stream: %+v", len(ts.tables), ts.tables) + log.Infof("Found %d tables to stream: %s", len(ts.tables), strings.Join(ts.tables, ", ")) for _, tableName := range ts.tables { log.Infof("Streaming table %s", tableName) if err := ts.streamTable(ts.ctx, tableName); err != nil { diff --git a/go/vt/wrangler/vexec.go b/go/vt/wrangler/vexec.go index 45a0e139c24..0734fa7b593 100644 --- a/go/vt/wrangler/vexec.go +++ b/go/vt/wrangler/vexec.go @@ -387,9 +387,9 @@ func (wr *Wrangler) getWorkflowActionQuery(action string) (string, error) { return query, nil } -// canStartWorkflow validates that, for an atomic copy workflow, none of the streams are still in the copy phase. +// canRestartWorkflow validates that, for an atomic copy workflow, none of the streams are still in the copy phase. // Since we copy all tables in a single snapshot, we cannot restart a workflow which broke before all tables were copied. -func (wr *Wrangler) canStartWorkflow(ctx context.Context, workflow, keyspace string) error { +func (wr *Wrangler) canRestartWorkflow(ctx context.Context, workflow, keyspace string) error { res, err := wr.ShowWorkflow(ctx, workflow, keyspace) if err != nil { return err @@ -418,7 +418,7 @@ func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace, switch action { case "start": - err = wr.canStartWorkflow(ctx, workflow, keyspace) + err = wr.canRestartWorkflow(ctx, workflow, keyspace) if err != nil { return nil, err } From 342465334335012b688b99d9fd4bde4f0a80fa82 Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Wed, 6 Sep 2023 11:58:30 +0200 Subject: [PATCH 6/7] Update proto files Signed-off-by: Rohit Nayak --- .../proto/binlogdata/binlogdata_vtproto.pb.go | 74 +++++++++++++++++++ go/vt/proto/vtctldata/vtctldata_vtproto.pb.go | 2 + 2 files changed, 76 insertions(+) diff --git a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go index c3b86502979..379583b0354 100644 --- a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go +++ b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go @@ -345,6 +345,7 @@ func (m *RowEvent) CloneVT() *RowEvent { TableName: m.TableName, Keyspace: m.Keyspace, Shard: m.Shard, + Flags: m.Flags, } if rhs := m.RowChanges; rhs != nil { tmpContainer := make([]*RowChange, len(rhs)) @@ -704,6 +705,67 @@ func (m *VStreamRowsResponse) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *VStreamTablesRequest) CloneVT() *VStreamTablesRequest { + if m == nil { + return (*VStreamTablesRequest)(nil) + } + r := &VStreamTablesRequest{ + EffectiveCallerId: m.EffectiveCallerId.CloneVT(), + ImmediateCallerId: m.ImmediateCallerId.CloneVT(), + Target: m.Target.CloneVT(), + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *VStreamTablesRequest) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *VStreamTablesResponse) CloneVT() *VStreamTablesResponse { + if m == nil { + return (*VStreamTablesResponse)(nil) + } + r := &VStreamTablesResponse{ + TableName: m.TableName, + Gtid: m.Gtid, + Lastpk: m.Lastpk.CloneVT(), + } + if rhs := m.Fields; rhs != nil { + tmpContainer := make([]*query.Field, len(rhs)) + for k, v := range rhs { + tmpContainer[k] = v.CloneVT() + } + r.Fields = tmpContainer + } + if rhs := m.Pkfields; rhs != nil { + tmpContainer := make([]*query.Field, len(rhs)) + for k, v := range rhs { + tmpContainer[k] = v.CloneVT() + } + r.Pkfields = tmpContainer + } + if rhs := m.Rows; rhs != nil { + tmpContainer := make([]*query.Row, len(rhs)) + for k, v := range rhs { + tmpContainer[k] = v.CloneVT() + } + r.Rows = tmpContainer + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *VStreamTablesResponse) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *LastPKEvent) CloneVT() *LastPKEvent { if m == nil { return (*LastPKEvent)(nil) @@ -3058,11 +3120,23 @@ var vtprotoPool_VStreamTablesResponse = sync.Pool{ } func (m *VStreamTablesResponse) ResetVT() { + for _, mm := range m.Fields { + mm.Reset() + } + f0 := m.Fields[:0] + for _, mm := range m.Pkfields { + mm.Reset() + } + f1 := m.Pkfields[:0] for _, mm := range m.Rows { mm.ResetVT() } + f2 := m.Rows[:0] m.Lastpk.ReturnToVTPool() m.Reset() + m.Fields = f0 + m.Pkfields = f1 + m.Rows = f2 } func (m *VStreamTablesResponse) ReturnToVTPool() { if m != nil { diff --git a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go index effc0ddf947..b4f4d4a00f8 100644 --- a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go +++ b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go @@ -110,6 +110,7 @@ func (m *MaterializeSettings) CloneVT() *MaterializeSettings { OnDdl: m.OnDdl, DeferSecondaryKeys: m.DeferSecondaryKeys, TabletSelectionPreference: m.TabletSelectionPreference, + AtomicCopy: m.AtomicCopy, } if rhs := m.TableSettings; rhs != nil { tmpContainer := make([]*TableMaterializeSettings, len(rhs)) @@ -2555,6 +2556,7 @@ func (m *MoveTablesCreateRequest) CloneVT() *MoveTablesCreateRequest { DeferSecondaryKeys: m.DeferSecondaryKeys, AutoStart: m.AutoStart, NoRoutingRules: m.NoRoutingRules, + AtomicCopy: m.AtomicCopy, } if rhs := m.Cells; rhs != nil { tmpContainer := make([]string, len(rhs)) From 2647b712ff54b9888038c6fd1773a1cb1b3ba5ea Mon Sep 17 00:00:00 2001 From: Rohit Nayak Date: Wed, 6 Sep 2023 13:23:28 +0200 Subject: [PATCH 7/7] Make CloneVT() changes in vcopier_atomic.go to align with changes in https://github.com/vitessio/vitess/pull/13914 Signed-off-by: Rohit Nayak --- .../tabletmanager/vreplication/vcopier_atomic.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go index 98b46c3e2a0..6252690a629 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go @@ -152,12 +152,16 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings lastpk = nil // pkfields are only used for logging, so that we can monitor progress. pkfields = make([]*querypb.Field, len(resp.Pkfields)) - copy(pkfields, resp.Pkfields) + for _, f := range resp.Pkfields { + pkfields = append(pkfields, f.CloneVT()) + } fieldEvent := &binlogdatapb.FieldEvent{ TableName: tableName, } - fieldEvent.Fields = append(fieldEvent.Fields, resp.Fields...) + for _, f := range resp.Fields { + fieldEvent.Fields = append(fieldEvent.Fields, f.CloneVT()) + } tablePlan, err := state.plan.buildExecutionPlan(fieldEvent) if err != nil { return err @@ -191,7 +195,6 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings }, } log.Infof("copying table %s with lastpk %v", tableName, lastpkbv) - // Prepare a vcopierCopyTask for the current batch of work. currCh := make(chan *vcopierCopyTaskResult, 1) currT := newVCopierCopyTask(newVCopierCopyTaskArgs(resp.Rows, resp.Lastpk))