diff --git a/.github/workflows/cluster_endtoend_vreplication_multi_tenant.yml b/.github/workflows/cluster_endtoend_vreplication_multi_tenant.yml new file mode 100644 index 00000000000..0b682e602cb --- /dev/null +++ b/.github/workflows/cluster_endtoend_vreplication_multi_tenant.yml @@ -0,0 +1,172 @@ +# DO NOT MODIFY: THIS FILE IS GENERATED USING "make generate_ci_workflows" + +name: Cluster (vreplication_multi_tenant) +on: [push, pull_request] +concurrency: + group: format('{0}-{1}', ${{ github.ref }}, 'Cluster (vreplication_multi_tenant)') + cancel-in-progress: true + +permissions: read-all + +env: + LAUNCHABLE_ORGANIZATION: "vitess" + LAUNCHABLE_WORKSPACE: "vitess-app" + GITHUB_PR_HEAD_SHA: "${{ github.event.pull_request.head.sha }}" + +jobs: + build: + name: Run endtoend tests on Cluster (vreplication_multi_tenant) + runs-on: gh-hosted-runners-4cores-1 + + steps: + - name: Skip CI + run: | + if [[ "${{contains( github.event.pull_request.labels.*.name, 'Skip CI')}}" == "true" ]]; then + echo "skipping CI due to the 'Skip CI' label" + exit 1 + fi + + - name: Check if workflow needs to be skipped + id: skip-workflow + run: | + skip='false' + if [[ "${{github.event.pull_request}}" == "" ]] && [[ "${{github.ref}}" != "refs/heads/main" ]] && [[ ! "${{github.ref}}" =~ ^refs/heads/release-[0-9]+\.[0-9]$ ]] && [[ ! "${{github.ref}}" =~ "refs/tags/.*" ]]; then + skip='true' + fi + echo Skip ${skip} + echo "skip-workflow=${skip}" >> $GITHUB_OUTPUT + + PR_DATA=$(curl -s\ + -H "Authorization: token ${{ secrets.GITHUB_TOKEN }}" \ + -H "Accept: application/vnd.github.v3+json" \ + "https://api.github.com/repos/${{ github.repository }}/pulls/${{ github.event.pull_request.number }}") + draft=$(echo "$PR_DATA" | jq .draft -r) + echo "is_draft=${draft}" >> $GITHUB_OUTPUT + + - name: Check out code + if: steps.skip-workflow.outputs.skip-workflow == 'false' + uses: actions/checkout@v4 + + - name: Check for changes in relevant files + if: steps.skip-workflow.outputs.skip-workflow == 'false' + uses: dorny/paths-filter@v3.0.1 + id: changes + with: + token: '' + filters: | + end_to_end: + - 'go/**/*.go' + - 'go/vt/sidecardb/**/*.sql' + - 'go/test/endtoend/onlineddl/vrepl_suite/**' + - 'test.go' + - 'Makefile' + - 'build.env' + - 'go.sum' + - 'go.mod' + - 'proto/*.proto' + - 'tools/**' + - 'config/**' + - 'bootstrap.sh' + - '.github/workflows/cluster_endtoend_vreplication_multi_tenant.yml' + + - name: Set up Go + if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' + uses: actions/setup-go@v5 + with: + go-version: 1.22.1 + + - name: Set up python + if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' + uses: actions/setup-python@v5 + + - name: Tune the OS + if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' + run: | + # Limit local port range to not use ports that overlap with server side + # ports that we listen on. + sudo sysctl -w net.ipv4.ip_local_port_range="22768 65535" + # Increase the asynchronous non-blocking I/O. More information at https://dev.mysql.com/doc/refman/5.7/en/innodb-parameters.html#sysvar_innodb_use_native_aio + echo "fs.aio-max-nr = 1048576" | sudo tee -a /etc/sysctl.conf + sudo sysctl -p /etc/sysctl.conf + + - name: Get dependencies + if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' + run: | + + # Get key to latest MySQL repo + sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys A8D3785C + # Setup MySQL 8.0 + wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.29-1_all.deb + echo mysql-apt-config mysql-apt-config/select-server select mysql-8.0 | sudo debconf-set-selections + sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config* + sudo apt-get -qq update + # Install everything else we need, and configure + sudo apt-get -qq install -y mysql-server mysql-client make unzip g++ etcd curl git wget eatmydata xz-utils libncurses5 + + sudo service mysql stop + sudo service etcd stop + sudo ln -s /etc/apparmor.d/usr.sbin.mysqld /etc/apparmor.d/disable/ + sudo apparmor_parser -R /etc/apparmor.d/usr.sbin.mysqld + go mod download + + # install JUnit report formatter + go install github.com/vitessio/go-junit-report@HEAD + + - name: Setup launchable dependencies + if: steps.skip-workflow.outputs.is_draft == 'false' && steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' && github.base_ref == 'main' + run: | + # Get Launchable CLI installed. If you can, make it a part of the builder image to speed things up + pip3 install --user launchable~=1.0 > /dev/null + + # verify that launchable setup is all correct. + launchable verify || true + + # Tell Launchable about the build you are producing and testing + launchable record build --name "$GITHUB_RUN_ID" --no-commit-collection --source . + + - name: Run cluster endtoend test + if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' + timeout-minutes: 45 + run: | + # We set the VTDATAROOT to the /tmp folder to reduce the file path of mysql.sock file + # which musn't be more than 107 characters long. + export VTDATAROOT="/tmp/" + source build.env + + set -exo pipefail + + # Increase our open file descriptor limit as we could hit this + ulimit -n 65536 + cat <<-EOF>>./config/mycnf/mysql80.cnf + innodb_buffer_pool_dump_at_shutdown=OFF + innodb_buffer_pool_in_core_file=OFF + innodb_buffer_pool_load_at_startup=OFF + innodb_buffer_pool_size=64M + innodb_doublewrite=OFF + innodb_flush_log_at_trx_commit=0 + innodb_flush_method=O_DIRECT + innodb_numa_interleave=ON + innodb_adaptive_hash_index=OFF + sync_binlog=0 + sync_relay_log=0 + performance_schema=OFF + slow-query-log=OFF + EOF + + cat <<-EOF>>./config/mycnf/mysql80.cnf + binlog-transaction-compression=ON + EOF + + # run the tests however you normally do, then produce a JUnit XML file + eatmydata -- go run test.go -docker=false -follow -shard vreplication_multi_tenant | tee -a output.txt | go-junit-report -set-exit-code > report.xml + + - name: Print test output and Record test result in launchable if PR is not a draft + if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' && always() + run: | + if [[ "${{steps.skip-workflow.outputs.is_draft}}" == "false" ]]; then + # send recorded tests to launchable + launchable record tests --build "$GITHUB_RUN_ID" go-test . || true + fi + + # print test output + cat output.txt diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml b/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml index bf6c4b0c38c..a2fa2f43d6e 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml @@ -201,6 +201,12 @@ jobs: rm -f $PWD/bin/vtgate $PWD/bin/vttablet $PWD/bin/mysqlctl $PWD/bin/mysqlctld cp /tmp/vitess-build-current/bin/vtgate $PWD/bin/vtgate + + cp /tmp/vitess-build-other/bin/vtctld $PWD/bin + cp /tmp/vitess-build-other/bin/vtctldclient $PWD/bin + cp /tmp/vitess-build-other/bin/vtctl $PWD/bin + cp /tmp/vitess-build-other/bin/vtctlclient $PWD/bin + cp /tmp/vitess-build-other/bin/vttablet $PWD/bin/vttablet cp /tmp/vitess-build-other/bin/mysqlctl $PWD/bin/mysqlctl cp /tmp/vitess-build-other/bin/mysqlctld $PWD/bin/mysqlctld diff --git a/.gitignore b/.gitignore index 881e89890cc..b4df3c39b82 100644 --- a/.gitignore +++ b/.gitignore @@ -86,3 +86,6 @@ report # plan test output /go/vt/vtgate/planbuilder/testdata/plan_test* + +# mise files +.mise.toml diff --git a/go/cmd/vtctldclient/command/keyspace_routing_rules.go b/go/cmd/vtctldclient/command/keyspace_routing_rules.go new file mode 100644 index 00000000000..cfdece351a6 --- /dev/null +++ b/go/cmd/vtctldclient/command/keyspace_routing_rules.go @@ -0,0 +1,156 @@ +/* +Copyright 2024 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 command + +import ( + "errors" + "fmt" + "os" + "strings" + + "github.com/spf13/cobra" + + "vitess.io/vitess/go/cmd/vtctldclient/cli" + "vitess.io/vitess/go/json2" + + vschemapb "vitess.io/vitess/go/vt/proto/vschema" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" +) + +var ( + // ApplyKeyspaceRoutingRules makes an ApplyKeyspaceRoutingRules gRPC call to a vtctld. + ApplyKeyspaceRoutingRules = &cobra.Command{ + Use: "ApplyKeyspaceRoutingRules {--rules RULES | --rules-file RULES_FILE} [--cells=c1,c2,...] [--skip-rebuild] [--dry-run]", + Short: "Applies the provided keyspace routing rules.", + DisableFlagsInUseLine: true, + Args: cobra.NoArgs, + PreRunE: validateApplyKeyspaceRoutingRulesOptions, + RunE: commandApplyKeyspaceRoutingRules, + } + // GetKeyspaceRoutingRules makes a GetKeyspaceRoutingRules gRPC call to a vtctld. + GetKeyspaceRoutingRules = &cobra.Command{ + Use: "GetKeyspaceRoutingRules", + Short: "Displays the currently active keyspace routing rules.", + DisableFlagsInUseLine: true, + Args: cobra.NoArgs, + RunE: commandGetKeyspaceRoutingRules, + } +) + +func validateApplyKeyspaceRoutingRulesOptions(cmd *cobra.Command, args []string) error { + opts := applyKeyspaceRoutingRulesOptions + if (opts.Rules != "" && opts.RulesFilePath != "") || (opts.Rules == "" && opts.RulesFilePath == "") { + return errors.New("must pass exactly one of --rules or --rules-file") + } + return nil +} + +var applyKeyspaceRoutingRulesOptions = struct { + Rules string + RulesFilePath string + Cells []string + SkipRebuild bool + DryRun bool +}{} + +func commandApplyKeyspaceRoutingRules(cmd *cobra.Command, args []string) error { + opts := applyKeyspaceRoutingRulesOptions + cli.FinishedParsing(cmd) + + var rulesBytes []byte + if opts.RulesFilePath != "" { + data, err := os.ReadFile(opts.RulesFilePath) + if err != nil { + return err + } + rulesBytes = data + } else { + rulesBytes = []byte(opts.Rules) + } + + krr := &vschemapb.KeyspaceRoutingRules{} + if err := json2.Unmarshal(rulesBytes, &krr); err != nil { + return err + } + // Round-trip so that when we display the result it's readable. + data, err := cli.MarshalJSON(krr) + if err != nil { + return err + } + + if opts.DryRun { + fmt.Printf("[DRY RUN] Would have saved new KeyspaceRoutingRules object:\n%s\n", data) + + if opts.SkipRebuild { + fmt.Println("[DRY RUN] Would not have rebuilt VSchema graph, would have required operator to run RebuildVSchemaGraph for changes to take effect.") + } else { + fmt.Print("[DRY RUN] Would have rebuilt the VSchema graph") + if len(opts.Cells) == 0 { + fmt.Print(" in all cells\n") + } else { + fmt.Printf(" in the following cells: %s.\n", strings.Join(applyKeyspaceRoutingRulesOptions.Cells, ", ")) + } + } + + return nil + } + + _, err = client.ApplyKeyspaceRoutingRules(commandCtx, &vtctldatapb.ApplyKeyspaceRoutingRulesRequest{ + KeyspaceRoutingRules: krr, + SkipRebuild: opts.SkipRebuild, + RebuildCells: opts.Cells, + }) + if err != nil { + return err + } + + fmt.Printf("New KeyspaceRoutingRules object:\n%s\nIf this is not what you expected, check the input data (as JSON parsing will skip unexpected fields).\n", data) + + if opts.SkipRebuild { + fmt.Println("Skipping rebuild of VSchema graph as requested, you will need to run RebuildVSchemaGraph for the changes to take effect.") + } + + return nil +} + +func commandGetKeyspaceRoutingRules(cmd *cobra.Command, args []string) error { + cli.FinishedParsing(cmd) + + resp, err := client.GetKeyspaceRoutingRules(commandCtx, &vtctldatapb.GetKeyspaceRoutingRulesRequest{}) + if err != nil { + return err + } + + data, err := cli.MarshalJSON(resp.KeyspaceRoutingRules) + if err != nil { + return err + } + + fmt.Printf("%s\n", data) + + return nil +} + +func init() { + ApplyKeyspaceRoutingRules.Flags().StringVarP(&applyKeyspaceRoutingRulesOptions.Rules, "rules", "r", "", "Keyspace routing rules, specified as a string") + ApplyKeyspaceRoutingRules.Flags().StringVarP(&applyKeyspaceRoutingRulesOptions.RulesFilePath, "rules-file", "f", "", "Path to a file containing keyspace routing rules specified as JSON") + ApplyKeyspaceRoutingRules.Flags().StringSliceVarP(&applyKeyspaceRoutingRulesOptions.Cells, "cells", "c", nil, "Limit the VSchema graph rebuilding to the specified cells. Ignored if --skip-rebuild is specified.") + ApplyKeyspaceRoutingRules.Flags().BoolVar(&applyKeyspaceRoutingRulesOptions.SkipRebuild, "skip-rebuild", false, "Skip rebuilding the SrvVSchema objects.") + ApplyKeyspaceRoutingRules.Flags().BoolVarP(&applyKeyspaceRoutingRulesOptions.DryRun, "dry-run", "d", false, "Validate the specified keyspace routing rules and note actions that would be taken, but do not actually apply the rules to the topo.") + Root.AddCommand(ApplyKeyspaceRoutingRules) + Root.AddCommand(GetKeyspaceRoutingRules) +} diff --git a/go/cmd/vtctldclient/command/vreplication/movetables/create.go b/go/cmd/vtctldclient/command/vreplication/movetables/create.go index 95c50f4f97e..22e9c5a92ce 100644 --- a/go/cmd/vtctldclient/command/vreplication/movetables/create.go +++ b/go/cmd/vtctldclient/command/vreplication/movetables/create.go @@ -39,6 +39,7 @@ var ( SourceTimeZone string NoRoutingRules bool AtomicCopy bool + WorkflowOptions vtctldatapb.WorkflowOptions }{} // create makes a MoveTablesCreate gRPC call to a vtctld. @@ -109,6 +110,7 @@ func commandCreate(cmd *cobra.Command, args []string) error { StopAfterCopy: common.CreateOptions.StopAfterCopy, NoRoutingRules: createOptions.NoRoutingRules, AtomicCopy: createOptions.AtomicCopy, + WorkflowOptions: &createOptions.WorkflowOptions, } resp, err := common.GetClient().MoveTablesCreate(common.GetCommandCtx(), req) diff --git a/go/cmd/vtctldclient/command/vreplication/movetables/movetables.go b/go/cmd/vtctldclient/command/vreplication/movetables/movetables.go index 0757fc542ac..9911756cb2a 100644 --- a/go/cmd/vtctldclient/command/vreplication/movetables/movetables.go +++ b/go/cmd/vtctldclient/command/vreplication/movetables/movetables.go @@ -47,6 +47,8 @@ func registerCommands(root *cobra.Command) { create.Flags().StringSliceVar(&createOptions.ExcludeTables, "exclude-tables", nil, "Source tables to exclude from copying.") create.Flags().BoolVar(&createOptions.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.") create.Flags().BoolVar(&createOptions.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.") + create.Flags().StringVar(&createOptions.WorkflowOptions.TenantId, "tenant-id", "", "(EXPERIMENTAL) The tenant ID to use for the MoveTables workflow into a multi-tenant keyspace.") + create.Flags().StringVar(&createOptions.WorkflowOptions.SourceKeyspaceAlias, "source-keyspace-alias", "", "(EXPERIMENTAL) Used currently only for multi-tenant migrations. This value will be used instead of the source keyspace name in the keyspace routing rules.") base.AddCommand(create) opts := &common.SubCommandsOpts{ diff --git a/go/flags/endtoend/vtctldclient.txt b/go/flags/endtoend/vtctldclient.txt index a37bbef1cc2..393b9ada10d 100644 --- a/go/flags/endtoend/vtctldclient.txt +++ b/go/flags/endtoend/vtctldclient.txt @@ -11,6 +11,7 @@ Usage: Available Commands: AddCellInfo Registers a local topology service in a new cell by creating the CellInfo. AddCellsAlias Defines a group of cells that can be referenced by a single name (the alias). + ApplyKeyspaceRoutingRules Applies the provided keyspace routing rules. ApplyRoutingRules Applies the VSchema routing rules. ApplySchema Applies the schema change to the specified keyspace on every primary, running in parallel on all shards. The changes are then propagated to replicas via replication. ApplyShardRoutingRules Applies the provided shard routing rules. @@ -39,6 +40,7 @@ Available Commands: GetCellsAliases Gets all CellsAlias objects in the cluster. GetFullStatus Outputs a JSON structure that contains full status of MySQL including the replication information, semi-sync information, GTID information among others. GetKeyspace Returns information about the given keyspace from the topology. + GetKeyspaceRoutingRules Displays the currently active keyspace routing rules. GetKeyspaces Returns information about every keyspace in the topology. GetPermissions Displays the permissions for a tablet. GetRoutingRules Displays the VSchema routing rules. diff --git a/go/test/endtoend/cluster/vtctldclient_process.go b/go/test/endtoend/cluster/vtctldclient_process.go index 57cb0cc4f45..2c6d6028ee0 100644 --- a/go/test/endtoend/cluster/vtctldclient_process.go +++ b/go/test/endtoend/cluster/vtctldclient_process.go @@ -22,6 +22,8 @@ import ( "strings" "time" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/json2" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/vterrors" @@ -69,7 +71,8 @@ func (vtctldclient *VtctldClientProcess) ExecuteCommandWithOutput(args ...string vtctldclient.Binary, filterDoubleDashArgs(pArgs, vtctldclient.VtctldClientMajorVersion)..., ) - log.Infof("Executing vtctldclient with command: %v (attempt %d of %d)", strings.Join(tmpProcess.Args, " "), i, retries) + msg := binlogplayer.LimitString(strings.Join(tmpProcess.Args, " "), 256) // limit log line length + log.Infof("Executing vtctldclient with command: %v (attempt %d of %d)", msg, i, retries) resultByte, err = tmpProcess.CombinedOutput() resultStr = string(resultByte) if err == nil || !shouldRetry(resultStr) { diff --git a/go/test/endtoend/vreplication/cluster_test.go b/go/test/endtoend/vreplication/cluster_test.go index 1863cba2b60..b88e919e5ca 100644 --- a/go/test/endtoend/vreplication/cluster_test.go +++ b/go/test/endtoend/vreplication/cluster_test.go @@ -747,7 +747,6 @@ func (vc *VitessCluster) AddShards(t *testing.T, cells []*Cell, keyspace *Keyspa } } log.Infof("Throttler config applied on all shards") - return nil } diff --git a/go/test/endtoend/vreplication/multi_tenant_test.go b/go/test/endtoend/vreplication/multi_tenant_test.go new file mode 100644 index 00000000000..ea1ee6511a1 --- /dev/null +++ b/go/test/endtoend/vreplication/multi_tenant_test.go @@ -0,0 +1,463 @@ +/* +Copyright 2024 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. +*/ + +/* +Consists of two tests. Both tests are for multi-tenant migration scenarios. + +1. TestMultiTenantSimple: migrates a single tenant to a target keyspace. + +2. TestMultiTenantComplex: migrates multiple tenants to a single target keyspace, with concurrent migrations. + +The tests use the MoveTables workflow to migrate the tenants. They are designed to simulate a real-world multi-tenant +migration scenario, where each tenant is in a separate database. +*/ + +package vreplication + +import ( + "encoding/json" + "fmt" + "math/rand/v2" + "strconv" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/vt/log" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" +) + +type tenantMigrationStatus int + +const ( + tenantMigrationStatusNotMigrated tenantMigrationStatus = iota + tenantMigrationStatusMigrating + tenantMigrationStatusMigrated + + sourceKeyspaceTemplate = "s%d" + sourceAliasKeyspaceTemplate = "a%d" + targetKeyspaceName = "mt" + + numTenants = 10 + numInitialRowsPerTenant = 10 + numAdditionalRowsPerTenant = 10 + baseInitialTabletId = 1000 + tabletIdStep = 100 + maxRandomDelaySeconds = 5 + waitTimeout = 10 * time.Minute +) + +var ( + // channels to coordinate the migration workflow + chNotSetup, chNotCreated, chInProgress, chSwitched, chCompleted chan int64 + // counters to keep track of the number of tenants in each state + numSetup, numInProgress, numSwitched, numCompleted atomic.Int64 +) + +// multiTenantMigration manages the migration of multiple tenants to a single target keyspace. +// A singleton object of this type is created for the test case. +type multiTenantMigration struct { + t *testing.T + mu sync.Mutex + tenantMigrationStatus map[int64]tenantMigrationStatus // current migration status for each tenant + activeMoveTables map[int64]*VtctldMoveTables // the internal MoveTables object for each tenant + + targetKeyspace string + tables string + tenantIdColumnName string // the name of the column in each table that holds the tenant ID + + lastIDs map[int64]int64 // the last primary key inserted for each tenant +} + +const ( + mtSchema = "create table t1(id int, tenant_id int, primary key(id, tenant_id)) Engine=InnoDB" + mtVSchema = ` +{ + "multi_tenant_spec": { + "tenant_id_column_name": "tenant_id", + "tenant_id_column_type": "INT64" + }, + "tables": { + "t1": {} + } +} +` + stSchema = mtSchema + stVSchema = ` +{ + "tables": { + "t1": {} + } +} +` +) + +// TestMultiTenantSimple tests a single tenant migration. The aim here is to test all the steps of the migration process +// including keyspace routing rules, addition of tenant filters to the forward and reverse vreplication streams, and +// verifying that the data is migrated correctly. +func TestMultiTenantSimple(t *testing.T) { + setSidecarDBName("_vt") + // Don't create RDONLY tablets to reduce number of tablets created to reduce resource requirements for the test. + origDefaultRdonly := defaultRdonly + defer func() { + defaultRdonly = origDefaultRdonly + }() + defaultRdonly = 0 + vc = setupMinimalCluster(t) + defer vc.TearDown() + + targetKeyspace := "mt" + _, err := vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, targetKeyspace, "0", mtVSchema, mtSchema, 1, 0, 200, nil) + require.NoError(t, err) + + tenantId := int64(1) + sourceKeyspace := getSourceKeyspace(tenantId) + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + _, err = vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, sourceKeyspace, "0", stVSchema, stSchema, 1, 0, getInitialTabletIdForTenant(tenantId), nil) + require.NoError(t, err) + + targetPrimary := vc.getPrimaryTablet(t, targetKeyspace, "0") + sourcePrimary := vc.getPrimaryTablet(t, sourceKeyspace, "0") + primaries := map[string]*cluster.VttabletProcess{ + "target": targetPrimary, + "source": sourcePrimary, + } + + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + numRows := 10 + lastIndex := int64(0) + insertRows := func(lastIndex int64, keyspace string) int64 { + for i := 1; i <= numRows; i++ { + execQueryWithRetry(t, vtgateConn, + fmt.Sprintf("insert into %s.t1(id, tenant_id) values(%d, %d)", keyspace, int64(i)+lastIndex, tenantId), queryTimeout) + } + return int64(numRows) + lastIndex + } + lastIndex = insertRows(lastIndex, sourceKeyspace) + + mt := newVtctldMoveTables(&moveTablesWorkflow{ + workflowInfo: &workflowInfo{ + vc: vc, + workflowName: fmt.Sprintf("wf%d", tenantId), + targetKeyspace: targetKeyspace, + }, + sourceKeyspace: sourceKeyspace, + createFlags: []string{ + "--tenant-id", strconv.FormatInt(tenantId, 10), + "--source-keyspace-alias", sourceAliasKeyspace, + }, + }) + + preSwitchRules := &vschemapb.KeyspaceRoutingRules{ + Rules: []*vschemapb.KeyspaceRoutingRule{ + {FromKeyspace: "a1", ToKeyspace: "s1"}, + {FromKeyspace: "s1", ToKeyspace: "s1"}, + }, + } + postSwitchRules := &vschemapb.KeyspaceRoutingRules{ + Rules: []*vschemapb.KeyspaceRoutingRule{ + {FromKeyspace: "a1", ToKeyspace: "mt"}, + {FromKeyspace: "s1", ToKeyspace: "mt"}, + }, + } + rulesMap := map[string]*vschemapb.KeyspaceRoutingRules{ + "pre": preSwitchRules, + "post": postSwitchRules, + } + require.Zero(t, len(getKeyspaceRoutingRules(t, vc).Rules)) + mt.Create() + validateKeyspaceRoutingRules(t, vc, primaries, rulesMap, false) + // Note: we cannot insert into the target keyspace since that is never routed to the source keyspace. + for _, ks := range []string{sourceKeyspace, sourceAliasKeyspace} { + lastIndex = insertRows(lastIndex, ks) + } + mt.SwitchReadsAndWrites() + validateKeyspaceRoutingRules(t, vc, primaries, rulesMap, true) + // Note: here we have already switched and we can insert into the target keyspace and it should get reverse + // replicated to the source keyspace. The source and alias are also routed to the target keyspace at this point. + for _, ks := range []string{sourceKeyspace, sourceAliasKeyspace, targetKeyspace} { + lastIndex = insertRows(lastIndex, ks) + } + mt.Complete() + require.Zero(t, len(getKeyspaceRoutingRules(t, vc).Rules)) + actualRowsInserted := getRowCount(t, vtgateConn, fmt.Sprintf("%s.%s", targetKeyspace, "t1")) + log.Infof("Migration completed, total rows in target: %d", actualRowsInserted) + require.Equal(t, lastIndex, int64(actualRowsInserted)) +} + +// If switched queries with source/alias qualifiers should execute on target, else on source. Confirm that +// the routing rules are as expected and that the query executes on the expected tablet. +func validateKeyspaceRoutingRules(t *testing.T, vc *VitessCluster, primaries map[string]*cluster.VttabletProcess, rulesMap map[string]*vschemapb.KeyspaceRoutingRules, switched bool) { + currentRules := getKeyspaceRoutingRules(t, vc) + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + queryTemplate := "select count(*) from %s.t1" + matchQuery := "select count(*) from t1" + + validateQueryRoute := func(qualifier, dest string) { + query := fmt.Sprintf(queryTemplate, qualifier) + assertQueryExecutesOnTablet(t, vtgateConn, primaries[dest], "", query, matchQuery) + log.Infof("query %s executed on %s", query, dest) + } + + if switched { + require.ElementsMatch(t, rulesMap["post"].Rules, currentRules.Rules) + validateQueryRoute("mt", "target") + validateQueryRoute("s1", "target") + validateQueryRoute("a1", "target") + } else { + require.ElementsMatch(t, rulesMap["pre"].Rules, currentRules.Rules) + // Note that with multi-tenant migration, we cannot redirect the target keyspace since + // there are multiple source keyspaces and the target has the aggregate of all the tenants. + validateQueryRoute("mt", "target") + validateQueryRoute("s1", "source") + validateQueryRoute("a1", "source") + } +} + +func getSourceKeyspace(tenantId int64) string { + return fmt.Sprintf(sourceKeyspaceTemplate, tenantId) +} + +func getSourceAliasKeyspace(tenantId int64) string { + return fmt.Sprintf(sourceAliasKeyspaceTemplate, tenantId) +} + +func (mtm *multiTenantMigration) insertSomeData(t *testing.T, tenantId int64, keyspace string, numRows int64) { + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + idx := mtm.getLastID(tenantId) + for i := idx + 1; i <= idx+numRows; i++ { + execQueryWithRetry(t, vtgateConn, + fmt.Sprintf("insert into %s.t1(id, tenant_id) values(%d, %d)", keyspace, i, tenantId), queryTimeout) + } + mtm.setLastID(tenantId, idx+numRows) +} + +func getKeyspaceRoutingRules(t *testing.T, vc *VitessCluster) *vschemapb.KeyspaceRoutingRules { + output, err := vc.VtctldClient.ExecuteCommandWithOutput("GetKeyspaceRoutingRules") + require.NoError(t, err) + rules := &vschemapb.KeyspaceRoutingRules{} + err = json.Unmarshal([]byte(output), rules) + require.NoError(t, err) + return rules +} + +// TestMultiTenant tests a multi-tenant migration scenario where each tenant is in a separate database. +// It uses MoveTables to migrate all tenants to the same target keyspace. The test creates a separate source keyspace +// for each tenant. It then steps through the migration process for each tenant, and verifies that the data is migrated +// correctly. The migration steps are done concurrently and randomly to simulate an actual multi-tenant migration. +func TestMultiTenantComplex(t *testing.T) { + setSidecarDBName("_vt") + // Don't create RDONLY tablets to reduce number of tablets created to reduce resource requirements for the test. + origDefaultRdonly := defaultRdonly + defer func() { + defaultRdonly = origDefaultRdonly + }() + defaultRdonly = 0 + vc = setupMinimalCluster(t) + defer vc.TearDown() + + mtm := newMultiTenantMigration(t) + numTenantsMigrated := 0 + mtm.run() // Start the migration process for all tenants. + timer := time.NewTimer(waitTimeout) + for numTenantsMigrated < numTenants { + select { + case tenantId := <-chCompleted: + mtm.setTenantMigrationStatus(tenantId, tenantMigrationStatusMigrated) + numTenantsMigrated++ + timer.Reset(waitTimeout) + case <-timer.C: + require.FailNow(t, "Timed out waiting for all tenants to complete") + } + } + vtgateConn, closeConn := getVTGateConn() + defer closeConn() + t.Run("Verify all rows have been migrated", func(t *testing.T) { + numAdditionalInsertSets := 2 // during the SwitchTraffic stop + totalRowsInsertedPerTenant := numInitialRowsPerTenant + numAdditionalRowsPerTenant*numAdditionalInsertSets + totalRowsInserted := totalRowsInsertedPerTenant * numTenants + totalActualRowsInserted := getRowCount(t, vtgateConn, fmt.Sprintf("%s.%s", mtm.targetKeyspace, "t1")) + require.Equal(t, totalRowsInserted, totalActualRowsInserted) + log.Infof("Migration completed, total rows inserted in target: %d", totalActualRowsInserted) + }) +} + +func newMultiTenantMigration(t *testing.T) *multiTenantMigration { + _, err := vc.AddKeyspace(t, []*Cell{vc.Cells["zone1"]}, targetKeyspaceName, "0", mtVSchema, mtSchema, 1, 0, 200, nil) + require.NoError(t, err) + mtm := &multiTenantMigration{ + t: t, + tenantMigrationStatus: make(map[int64]tenantMigrationStatus), + activeMoveTables: make(map[int64]*VtctldMoveTables), + targetKeyspace: targetKeyspaceName, + tables: "t1", + tenantIdColumnName: "tenant_id", + lastIDs: make(map[int64]int64), + } + for i := 1; i <= numTenants; i++ { + mtm.setTenantMigrationStatus(int64(i), tenantMigrationStatusNotMigrated) + } + channelSize := numTenants + 1 // +1 to make sure the channels never block + for _, ch := range []*chan int64{&chNotSetup, &chNotCreated, &chInProgress, &chSwitched, &chCompleted} { + *ch = make(chan int64, channelSize) + } + return mtm +} + +func (mtm *multiTenantMigration) getTenantMigrationStatus(tenantId int64) tenantMigrationStatus { + mtm.mu.Lock() + defer mtm.mu.Unlock() + return mtm.tenantMigrationStatus[tenantId] +} + +func (mtm *multiTenantMigration) setTenantMigrationStatus(tenantId int64, status tenantMigrationStatus) { + mtm.mu.Lock() + defer mtm.mu.Unlock() + mtm.tenantMigrationStatus[tenantId] = status +} + +func (mtm *multiTenantMigration) getActiveMoveTables(tenantId int64) *VtctldMoveTables { + mtm.mu.Lock() + defer mtm.mu.Unlock() + return mtm.activeMoveTables[tenantId] +} + +func (mtm *multiTenantMigration) setActiveMoveTables(tenantId int64, mt *VtctldMoveTables) { + mtm.mu.Lock() + defer mtm.mu.Unlock() + mtm.activeMoveTables[tenantId] = mt +} + +func (mtm *multiTenantMigration) setLastID(tenantId, lastID int64) { + mtm.mu.Lock() + defer mtm.mu.Unlock() + mtm.lastIDs[tenantId] = lastID +} + +func (mtm *multiTenantMigration) getLastID(tenantId int64) int64 { + mtm.mu.Lock() + defer mtm.mu.Unlock() + return mtm.lastIDs[tenantId] +} + +func (mtm *multiTenantMigration) initTenantData(t *testing.T, tenantId int64, sourceAliasKeyspace string) { + mtm.insertSomeData(t, tenantId, getSourceKeyspace(tenantId), numInitialRowsPerTenant) +} + +func getInitialTabletIdForTenant(tenantId int64) int { + return int(baseInitialTabletId + tenantId*tabletIdStep) +} + +func (mtm *multiTenantMigration) setup(tenantId int64) { + log.Infof("Creating MoveTables for tenant %d", tenantId) + mtm.setLastID(tenantId, 0) + sourceKeyspace := getSourceKeyspace(tenantId) + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + _, err := vc.AddKeyspace(mtm.t, []*Cell{vc.Cells["zone1"]}, sourceKeyspace, "0", stVSchema, stSchema, + 1, 0, getInitialTabletIdForTenant(tenantId), nil) + require.NoError(mtm.t, err) + mtm.initTenantData(mtm.t, tenantId, sourceAliasKeyspace) +} + +func (mtm *multiTenantMigration) start(tenantId int64) { + sourceKeyspace := getSourceKeyspace(tenantId) + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + mtm.setTenantMigrationStatus(tenantId, tenantMigrationStatusMigrating) + mt := newVtctldMoveTables(&moveTablesWorkflow{ + workflowInfo: &workflowInfo{ + vc: vc, + workflowName: fmt.Sprintf("wf%d", tenantId), + targetKeyspace: mtm.targetKeyspace, + }, + sourceKeyspace: sourceKeyspace, + tables: mtm.tables, + createFlags: []string{ + "--tenant-id", strconv.FormatInt(tenantId, 10), + "--source-keyspace-alias", sourceAliasKeyspace, + }, + }) + mtm.setActiveMoveTables(tenantId, mt) + mt.Create() +} + +func (mtm *multiTenantMigration) switchTraffic(tenantId int64) { + t := mtm.t + sourceAliasKeyspace := getSourceAliasKeyspace(tenantId) + sourceKeyspaceName := getSourceKeyspace(tenantId) + mt := mtm.getActiveMoveTables(tenantId) + ksWorkflow := fmt.Sprintf("%s.%s", mtm.targetKeyspace, mt.workflowName) + waitForWorkflowState(t, vc, ksWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String()) + // we intentionally insert first into the source alias keyspace and then the source keyspace to test routing rules for both. + mtm.insertSomeData(t, tenantId, sourceAliasKeyspace, numAdditionalRowsPerTenant) + mt.SwitchReadsAndWrites() + mtm.insertSomeData(t, tenantId, sourceKeyspaceName, numAdditionalRowsPerTenant) +} + +func (mtm *multiTenantMigration) complete(tenantId int64) { + mt := mtm.getActiveMoveTables(tenantId) + mt.Complete() + vtgateConn := vc.GetVTGateConn(mtm.t) + defer vtgateConn.Close() + waitForQueryResult(mtm.t, vtgateConn, "", + fmt.Sprintf("select count(*) from %s.t1 where tenant_id=%d", mt.targetKeyspace, tenantId), + fmt.Sprintf("[[INT64(%d)]]", mtm.getLastID(tenantId))) +} + +func randomWait() { + time.Sleep(time.Duration(rand.IntN(maxRandomDelaySeconds)) * time.Second) +} + +func (mtm *multiTenantMigration) doThis(name string, chIn, chOut chan int64, counter *atomic.Int64, f func(int64)) { + timer := time.NewTimer(waitTimeout) + for counter.Load() < numTenants { + select { + case tenantId := <-chIn: + f(tenantId) + counter.Add(1) + chOut <- tenantId + timer.Reset(waitTimeout) + case <-timer.C: + require.FailNowf(mtm.t, "Timed out: %s", name) + } + randomWait() + } +} + +// run starts the migration process for all tenants. It starts concurrent +func (mtm *multiTenantMigration) run() { + go mtm.doThis("Setup tenant keyspace/schemas", chNotSetup, chNotCreated, &numSetup, mtm.setup) + for i := int64(1); i <= numTenants; i++ { + chNotSetup <- i + } + // Wait for all tenants to be created before starting the workflows: 10 seconds per tenant to account for CI overhead. + perTenantLoadTimeout := 1 * time.Minute + require.NoError(mtm.t, waitForCondition("All tenants created", + func() bool { + return numSetup.Load() == numTenants + }, perTenantLoadTimeout*numTenants)) + + go mtm.doThis("Start Migrations", chNotCreated, chInProgress, &numInProgress, mtm.start) + go mtm.doThis("Switch Traffic", chInProgress, chSwitched, &numSwitched, mtm.switchTraffic) + go mtm.doThis("Mark Migrations Complete", chSwitched, chCompleted, &numCompleted, mtm.complete) +} diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index 0fa61d21d2c..ea2c9c63a51 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -618,10 +618,11 @@ func CreateVReplication(workflow string, source *binlogdatapb.BinlogSource, posi workflowType binlogdatapb.VReplicationWorkflowType, workflowSubType binlogdatapb.VReplicationWorkflowSubType, deferSecondaryKeys bool) string { protoutil.SortBinlogSourceTables(source) return fmt.Sprintf("insert into _vt.vreplication "+ - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) "+ - "values (%v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d, %v)", + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) "+ + "values (%v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d, %v, %s)", encodeString(workflow), encodeString(source.String()), encodeString(position), maxTPS, maxReplicationLag, - timeUpdated, binlogdatapb.VReplicationWorkflowState_Running.String(), encodeString(dbName), workflowType, workflowSubType, deferSecondaryKeys) + timeUpdated, binlogdatapb.VReplicationWorkflowState_Running.String(), encodeString(dbName), workflowType, + workflowSubType, deferSecondaryKeys, encodeString("{}")) } // CreateVReplicationState returns a statement to create a stopped vreplication. @@ -629,11 +630,11 @@ func CreateVReplicationState(workflow string, source *binlogdatapb.BinlogSource, workflowType binlogdatapb.VReplicationWorkflowType, workflowSubType binlogdatapb.VReplicationWorkflowSubType) string { protoutil.SortBinlogSourceTables(source) return fmt.Sprintf("insert into _vt.vreplication "+ - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type) "+ - "values (%v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d)", + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, options) "+ + "values (%v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d, %s)", encodeString(workflow), encodeString(source.String()), encodeString(position), throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), state.String(), encodeString(dbName), - workflowType, workflowSubType) + workflowType, workflowSubType, encodeString("{}")) } // GenerateUpdatePos returns a statement to record the latest processed gtid in the _vt.vreplication table. diff --git a/go/vt/binlog/binlogplayer/binlog_player_test.go b/go/vt/binlog/binlogplayer/binlog_player_test.go index 148c4fb386b..5c6e28df704 100644 --- a/go/vt/binlog/binlogplayer/binlog_player_test.go +++ b/go/vt/binlog/binlogplayer/binlog_player_test.go @@ -381,8 +381,8 @@ func applyEvents(blp *BinlogPlayer) func() error { func TestCreateVReplicationKeyRange(t *testing.T) { want := "insert into _vt.vreplication " + - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) " + - `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" key_range:{end:\"\\x80\"}', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running', 'db', 0, 0, false)` + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) " + + `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" key_range:{end:\"\\x80\"}', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running', 'db', 0, 0, false, '{}')` bls := binlogdatapb.BinlogSource{ Keyspace: "ks", @@ -400,8 +400,8 @@ func TestCreateVReplicationKeyRange(t *testing.T) { func TestCreateVReplicationTables(t *testing.T) { want := "insert into _vt.vreplication " + - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) " + - `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" tables:\"a\" tables:\"b\"', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running', 'db', 0, 0, false)` + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) " + + `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" tables:\"a\" tables:\"b\"', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running', 'db', 0, 0, false, '{}')` bls := binlogdatapb.BinlogSource{ Keyspace: "ks", diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 270e3ddfe51..2cdfa10a9bf 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -4958,7 +4958,8 @@ type CreateVReplicationWorkflowRequest struct { // AutoStart specifies if the workflow should be started when created. AutoStart bool `protobuf:"varint,9,opt,name=auto_start,json=autoStart,proto3" json:"auto_start,omitempty"` // Should the workflow stop after the copy phase. - StopAfterCopy bool `protobuf:"varint,10,opt,name=stop_after_copy,json=stopAfterCopy,proto3" json:"stop_after_copy,omitempty"` + StopAfterCopy bool `protobuf:"varint,10,opt,name=stop_after_copy,json=stopAfterCopy,proto3" json:"stop_after_copy,omitempty"` + Options string `protobuf:"bytes,11,opt,name=options,proto3" json:"options,omitempty"` } func (x *CreateVReplicationWorkflowRequest) Reset() { @@ -5063,6 +5064,13 @@ func (x *CreateVReplicationWorkflowRequest) GetStopAfterCopy() bool { return false } +func (x *CreateVReplicationWorkflowRequest) GetOptions() string { + if x != nil { + return x.Options + } + return "" +} + type CreateVReplicationWorkflowResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5485,6 +5493,7 @@ type ReadVReplicationWorkflowResponse struct { WorkflowSubType binlogdata.VReplicationWorkflowSubType `protobuf:"varint,9,opt,name=workflow_sub_type,json=workflowSubType,proto3,enum=binlogdata.VReplicationWorkflowSubType" json:"workflow_sub_type,omitempty"` DeferSecondaryKeys bool `protobuf:"varint,10,opt,name=defer_secondary_keys,json=deferSecondaryKeys,proto3" json:"defer_secondary_keys,omitempty"` Streams []*ReadVReplicationWorkflowResponse_Stream `protobuf:"bytes,11,rep,name=streams,proto3" json:"streams,omitempty"` + Options string `protobuf:"bytes,12,opt,name=options,proto3" json:"options,omitempty"` } func (x *ReadVReplicationWorkflowResponse) Reset() { @@ -5589,6 +5598,13 @@ func (x *ReadVReplicationWorkflowResponse) GetStreams() []*ReadVReplicationWorkf return nil } +func (x *ReadVReplicationWorkflowResponse) GetOptions() string { + if x != nil { + return x.Options + } + return "" +} + type VDiffRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7206,7 +7222,7 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x01, 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, 0xd4, 0x04, 0x0a, 0x21, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x56, + 0x76, 0x65, 0x6e, 0x74, 0x22, 0xee, 0x04, 0x0a, 0x21, 0x43, 0x72, 0x65, 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, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, @@ -7243,289 +7259,293 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 0x72, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x74, - 0x6f, 0x70, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 0x22, 0x50, 0x0a, 0x22, 0x43, - 0x72, 0x65, 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, 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, 0x3f, 0x0a, - 0x21, 0x44, 0x65, 0x6c, 0x65, 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, 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, 0x22, 0x50, - 0x0a, 0x22, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 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, 0x21, 0x0a, 0x1f, 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x34, 0x0a, 0x20, 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x68, 0x61, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x68, 0x61, 0x73, 0x22, 0xe0, 0x02, 0x0a, 0x20, 0x52, 0x65, - 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, - 0x0a, 0x0b, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x05, 0x52, 0x0a, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, - 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x4c, 0x0a, 0x0e, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, - 0x61, 0x2e, 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, 0x52, 0x0d, 0x69, 0x6e, 0x63, - 0x6c, 0x75, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, - 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, - 0x64, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0e, 0x32, - 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x5f, 0x66, 0x72, 0x6f, 0x7a, 0x65, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, - 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x46, 0x72, 0x6f, 0x7a, 0x65, 0x6e, 0x22, 0x76, 0x0a, 0x21, - 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x6f, 0x70, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x6f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x22, 0x43, 0x72, 0x65, 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, 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, 0x3f, 0x0a, 0x21, 0x44, 0x65, 0x6c, 0x65, 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, 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, 0x22, 0x50, 0x0a, 0x22, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 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, 0x21, 0x0a, 0x1f, 0x48, 0x61, + 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x34, 0x0a, + 0x20, 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x51, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x3d, 0x0a, 0x1f, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, + 0x65, 0x12, 0x10, 0x0a, 0x03, 0x68, 0x61, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, + 0x68, 0x61, 0x73, 0x22, 0xe0, 0x02, 0x0a, 0x20, 0x52, 0x65, 0x61, 0x64, 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, 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, 0x22, 0x94, 0x09, 0x0a, 0x20, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 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, - 0x01, 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, 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, 0x05, 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, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, - 0x67, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x49, - 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, 0x11, 0x77, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, - 0x61, 0x2e, 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, 0x52, 0x0f, 0x77, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 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, 0x0a, 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, 0x54, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x07, 0x73, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x1a, 0xc1, 0x04, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, - 0x64, 0x12, 0x2a, 0x0a, 0x03, 0x62, 0x6c, 0x73, 0x18, 0x02, 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, 0x03, 0x62, 0x6c, 0x73, 0x12, 0x10, 0x0a, - 0x03, 0x70, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x70, 0x6f, 0x73, 0x12, - 0x19, 0x0a, 0x08, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x6d, 0x61, - 0x78, 0x5f, 0x74, 0x70, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6d, 0x61, 0x78, - 0x54, 0x70, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x11, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x4c, 0x61, 0x67, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x64, 0x18, 0x07, 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, 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, 0x08, 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, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, - 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, - 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, - 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x0b, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, - 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, - 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, - 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x48, 0x65, 0x61, 0x72, 0x74, - 0x62, 0x65, 0x61, 0x74, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x68, 0x72, - 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, - 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, - 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, - 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, - 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, - 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0xd7, 0x01, 0x0a, 0x0c, 0x56, - 0x44, 0x69, 0x66, 0x66, 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, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x72, 0x67, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, - 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x07, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6a, 0x0a, 0x0d, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, - 0x02, 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, 0x6f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, - 0x22, 0x79, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x61, - 0x72, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x22, 0x90, 0x01, 0x0a, 0x12, - 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, 0x6b, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x6b, 0x73, 0x12, 0x1f, 0x0a, - 0x0b, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0a, 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x16, - 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, - 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x61, - 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0d, 0x6d, 0x61, 0x78, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x22, 0xda, - 0x02, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, - 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, 0x65, 0x74, 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, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, - 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, - 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x63, 0x74, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x50, 0x63, 0x74, - 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x6f, - 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x38, 0x0a, 0x19, 0x6d, 0x61, 0x78, - 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x63, - 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6d, 0x61, - 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, - 0x61, 0x72, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, - 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x73, 0x65, - 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, - 0x44, 0x69, 0x66, 0x66, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xf2, 0x01, 0x0a, 0x0c, - 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, - 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, - 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, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, - 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0d, 0x70, 0x69, 0x63, - 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, 0x0a, 0x0c, 0x63, 0x6f, - 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 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, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 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, 0x56, - 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x81, 0x03, 0x0a, 0x21, 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, 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, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 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, 0x03, 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, 0x04, 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, - 0x2e, 0x0a, 0x06, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4f, 0x6e, 0x44, - 0x44, 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x6f, 0x6e, 0x44, 0x64, 0x6c, 0x12, - 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x6e, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x69, + 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, + 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, - 0x61, 0x72, 0x64, 0x73, 0x22, 0x50, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, + 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x10, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, + 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, + 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x66, 0x72, 0x6f, 0x7a, 0x65, + 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x46, 0x72, 0x6f, 0x7a, 0x65, 0x6e, 0x22, 0x76, 0x0a, 0x21, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 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, 0x9f, 0x02, 0x0a, 0x22, 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, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, - 0x0d, 0x61, 0x6c, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, - 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, - 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x65, 0x78, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, - 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 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, 0x22, 0x51, 0x0a, 0x23, 0x55, 0x70, 0x64, 0x61, + 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x51, 0x0a, 0x09, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x3d, + 0x0a, 0x1f, 0x52, 0x65, 0x61, 0x64, 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, 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, 0x22, 0xae, 0x09, + 0x0a, 0x20, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 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, 0x01, 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, 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, 0x05, 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, 0x17, 0x0a, 0x07, 0x64, + 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x49, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x24, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, + 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 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, 0x0a, 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, 0x54, 0x0a, 0x07, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, + 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0xc1, 0x04, 0x0a, 0x06, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x03, 0x62, 0x6c, 0x73, 0x18, 0x02, 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, 0x03, 0x62, 0x6c, + 0x73, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x70, 0x6f, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x12, 0x17, + 0x0a, 0x07, 0x6d, 0x61, 0x78, 0x5f, 0x74, 0x70, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x06, 0x6d, 0x61, 0x78, 0x54, 0x70, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x5f, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, + 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x18, 0x07, 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, 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, 0x08, 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, 0x3b, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, + 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, + 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, + 0x69, 0x65, 0x64, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, + 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, + 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x48, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, + 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, + 0x74, 0x69, 0x6d, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, + 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, + 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0xd7, + 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, 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, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x72, 0x67, 0x12, 0x1d, + 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, 0x12, 0x39, 0x0a, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6a, 0x0a, 0x0d, 0x56, 0x44, 0x69, 0x66, + 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x6f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x18, 0x02, 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, 0x6f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, + 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, + 0x55, 0x75, 0x69, 0x64, 0x22, 0x79, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, + 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, + 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x1f, + 0x0a, 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x22, + 0x90, 0x01, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, + 0x6b, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x6b, + 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, + 0x78, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, + 0x77, 0x73, 0x22, 0xda, 0x02, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, + 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, 0x65, 0x74, 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, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, + 0x70, 0x63, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, + 0x65, 0x50, 0x63, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x74, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x38, 0x0a, + 0x19, 0x6d, 0x61, 0x78, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, + 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x15, 0x6d, 0x61, 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, + 0x66, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0e, 0x6d, 0x61, 0x78, 0x44, 0x69, 0x66, 0x66, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, + 0xf2, 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x4c, 0x0a, 0x0e, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x01, 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, 0x56, 0x44, 0x69, + 0x66, 0x66, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x0d, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, + 0x0a, 0x0c, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, + 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, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, + 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x65, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, + 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 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, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x81, 0x03, 0x0a, 0x21, 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, 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, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, + 0x02, 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, 0x03, 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, 0x04, 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, 0x2e, 0x0a, 0x06, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x4f, 0x6e, 0x44, 0x44, 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x6f, 0x6e, + 0x44, 0x64, 0x6c, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, + 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x22, 0x50, 0x0a, 0x22, 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, 0x73, 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, 0x2f, 0x0a, 0x15, 0x52, - 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, - 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x15, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, - 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x16, 0x43, - 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, - 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1c, 0x0a, 0x09, - 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, - 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, - 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3e, 0x0a, 0x19, 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, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4e, 0x59, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x49, - 0x4e, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, - 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x42, 0x30, 0x5a, 0x2e, 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, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 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, 0x9f, 0x02, 0x0a, 0x22, 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, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x61, 0x6c, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, + 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, + 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 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, 0x22, 0x51, 0x0a, 0x23, + 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, 0x73, 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, + 0x2f, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x22, 0x18, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, + 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x15, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc8, + 0x01, 0x0a, 0x16, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x14, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, + 0x0a, 0x10, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, + 0x6c, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3e, 0x0a, 0x19, 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, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4e, 0x59, 0x10, 0x00, 0x12, + 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, + 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x42, 0x30, 0x5a, 0x2e, 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, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go index 83cdb00115e..153c96c9c14 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go @@ -1891,6 +1891,7 @@ func (m *CreateVReplicationWorkflowRequest) CloneVT() *CreateVReplicationWorkflo DeferSecondaryKeys: m.DeferSecondaryKeys, AutoStart: m.AutoStart, StopAfterCopy: m.StopAfterCopy, + Options: m.Options, } if rhs := m.BinlogSource; rhs != nil { tmpContainer := make([]*binlogdata.BinlogSource, len(rhs)) @@ -2136,6 +2137,7 @@ func (m *ReadVReplicationWorkflowResponse) CloneVT() *ReadVReplicationWorkflowRe WorkflowType: m.WorkflowType, WorkflowSubType: m.WorkflowSubType, DeferSecondaryKeys: m.DeferSecondaryKeys, + Options: m.Options, } if rhs := m.TabletTypes; rhs != nil { tmpContainer := make([]topodata.TabletType, len(rhs)) @@ -6814,6 +6816,13 @@ func (m *CreateVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.Options) > 0 { + i -= len(m.Options) + copy(dAtA[i:], m.Options) + i = encodeVarint(dAtA, i, uint64(len(m.Options))) + i-- + dAtA[i] = 0x5a + } if m.StopAfterCopy { i-- if m.StopAfterCopy { @@ -7488,6 +7497,13 @@ func (m *ReadVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) ( i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if len(m.Options) > 0 { + i -= len(m.Options) + copy(dAtA[i:], m.Options) + i = encodeVarint(dAtA, i, uint64(len(m.Options))) + i-- + dAtA[i] = 0x62 + } if len(m.Streams) > 0 { for iNdEx := len(m.Streams) - 1; iNdEx >= 0; iNdEx-- { size, err := m.Streams[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) @@ -9977,6 +9993,10 @@ func (m *CreateVReplicationWorkflowRequest) SizeVT() (n int) { if m.StopAfterCopy { n += 2 } + l = len(m.Options) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -10230,6 +10250,10 @@ func (m *ReadVReplicationWorkflowResponse) SizeVT() (n int) { n += 1 + l + sov(uint64(l)) } } + l = len(m.Options) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -19830,6 +19854,38 @@ func (m *CreateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { } } m.StopAfterCopy = bool(v != 0) + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Options", 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.Options = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -21539,6 +21595,38 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Options", 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.Options = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) diff --git a/go/vt/proto/vschema/vschema.pb.go b/go/vt/proto/vschema/vschema.pb.go index 96f98128c6c..862c5271b52 100644 --- a/go/vt/proto/vschema/vschema.pb.go +++ b/go/vt/proto/vschema/vschema.pb.go @@ -211,6 +211,8 @@ type Keyspace struct { RequireExplicitRouting bool `protobuf:"varint,4,opt,name=require_explicit_routing,json=requireExplicitRouting,proto3" json:"require_explicit_routing,omitempty"` // foreign_key_mode dictates how Vitess should handle foreign keys for this keyspace. ForeignKeyMode Keyspace_ForeignKeyMode `protobuf:"varint,5,opt,name=foreign_key_mode,json=foreignKeyMode,proto3,enum=vschema.Keyspace_ForeignKeyMode" json:"foreign_key_mode,omitempty"` + // multi_tenant_mode specifies that the keyspace is multi-tenant. Currently used during migrations with MoveTables. + MultiTenantSpec *MultiTenantSpec `protobuf:"bytes,6,opt,name=multi_tenant_spec,json=multiTenantSpec,proto3" json:"multi_tenant_spec,omitempty"` } func (x *Keyspace) Reset() { @@ -280,6 +282,70 @@ func (x *Keyspace) GetForeignKeyMode() Keyspace_ForeignKeyMode { return Keyspace_unspecified } +func (x *Keyspace) GetMultiTenantSpec() *MultiTenantSpec { + if x != nil { + return x.MultiTenantSpec + } + return nil +} + +type MultiTenantSpec struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // tenant_column is the name of the column that specifies the tenant id. + TenantIdColumnName string `protobuf:"bytes,1,opt,name=tenant_id_column_name,json=tenantIdColumnName,proto3" json:"tenant_id_column_name,omitempty"` + // tenant_column_type is the type of the column that specifies the tenant id. + TenantIdColumnType query.Type `protobuf:"varint,2,opt,name=tenant_id_column_type,json=tenantIdColumnType,proto3,enum=query.Type" json:"tenant_id_column_type,omitempty"` +} + +func (x *MultiTenantSpec) Reset() { + *x = MultiTenantSpec{} + if protoimpl.UnsafeEnabled { + mi := &file_vschema_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MultiTenantSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MultiTenantSpec) ProtoMessage() {} + +func (x *MultiTenantSpec) ProtoReflect() protoreflect.Message { + mi := &file_vschema_proto_msgTypes[3] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MultiTenantSpec.ProtoReflect.Descriptor instead. +func (*MultiTenantSpec) Descriptor() ([]byte, []int) { + return file_vschema_proto_rawDescGZIP(), []int{3} +} + +func (x *MultiTenantSpec) GetTenantIdColumnName() string { + if x != nil { + return x.TenantIdColumnName + } + return "" +} + +func (x *MultiTenantSpec) GetTenantIdColumnType() query.Type { + if x != nil { + return x.TenantIdColumnType + } + return query.Type(0) +} + // Vindex is the vindex info for a Keyspace. type Vindex struct { state protoimpl.MessageState @@ -304,7 +370,7 @@ type Vindex struct { func (x *Vindex) Reset() { *x = Vindex{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[3] + mi := &file_vschema_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -317,7 +383,7 @@ func (x *Vindex) String() string { func (*Vindex) ProtoMessage() {} func (x *Vindex) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[3] + mi := &file_vschema_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -330,7 +396,7 @@ func (x *Vindex) ProtoReflect() protoreflect.Message { // Deprecated: Use Vindex.ProtoReflect.Descriptor instead. func (*Vindex) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{3} + return file_vschema_proto_rawDescGZIP(), []int{4} } func (x *Vindex) GetType() string { @@ -392,7 +458,7 @@ type Table struct { func (x *Table) Reset() { *x = Table{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[4] + mi := &file_vschema_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -405,7 +471,7 @@ func (x *Table) String() string { func (*Table) ProtoMessage() {} func (x *Table) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[4] + mi := &file_vschema_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -418,7 +484,7 @@ func (x *Table) ProtoReflect() protoreflect.Message { // Deprecated: Use Table.ProtoReflect.Descriptor instead. func (*Table) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{4} + return file_vschema_proto_rawDescGZIP(), []int{5} } func (x *Table) GetType() string { @@ -487,7 +553,7 @@ type ColumnVindex struct { func (x *ColumnVindex) Reset() { *x = ColumnVindex{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[5] + mi := &file_vschema_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -500,7 +566,7 @@ func (x *ColumnVindex) String() string { func (*ColumnVindex) ProtoMessage() {} func (x *ColumnVindex) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[5] + mi := &file_vschema_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -513,7 +579,7 @@ func (x *ColumnVindex) ProtoReflect() protoreflect.Message { // Deprecated: Use ColumnVindex.ProtoReflect.Descriptor instead. func (*ColumnVindex) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{5} + return file_vschema_proto_rawDescGZIP(), []int{6} } func (x *ColumnVindex) GetColumn() string { @@ -551,7 +617,7 @@ type AutoIncrement struct { func (x *AutoIncrement) Reset() { *x = AutoIncrement{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[6] + mi := &file_vschema_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -564,7 +630,7 @@ func (x *AutoIncrement) String() string { func (*AutoIncrement) ProtoMessage() {} func (x *AutoIncrement) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[6] + mi := &file_vschema_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -577,7 +643,7 @@ func (x *AutoIncrement) ProtoReflect() protoreflect.Message { // Deprecated: Use AutoIncrement.ProtoReflect.Descriptor instead. func (*AutoIncrement) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{6} + return file_vschema_proto_rawDescGZIP(), []int{7} } func (x *AutoIncrement) GetColumn() string { @@ -615,7 +681,7 @@ type Column struct { func (x *Column) Reset() { *x = Column{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[7] + mi := &file_vschema_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -628,7 +694,7 @@ func (x *Column) String() string { func (*Column) ProtoMessage() {} func (x *Column) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[7] + mi := &file_vschema_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -641,7 +707,7 @@ func (x *Column) ProtoReflect() protoreflect.Message { // Deprecated: Use Column.ProtoReflect.Descriptor instead. func (*Column) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{7} + return file_vschema_proto_rawDescGZIP(), []int{8} } func (x *Column) GetName() string { @@ -714,15 +780,16 @@ type SrvVSchema struct { unknownFields protoimpl.UnknownFields // keyspaces is a map of keyspace name -> Keyspace object. - Keyspaces map[string]*Keyspace `protobuf:"bytes,1,rep,name=keyspaces,proto3" json:"keyspaces,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - RoutingRules *RoutingRules `protobuf:"bytes,2,opt,name=routing_rules,json=routingRules,proto3" json:"routing_rules,omitempty"` // table routing rules - ShardRoutingRules *ShardRoutingRules `protobuf:"bytes,3,opt,name=shard_routing_rules,json=shardRoutingRules,proto3" json:"shard_routing_rules,omitempty"` + Keyspaces map[string]*Keyspace `protobuf:"bytes,1,rep,name=keyspaces,proto3" json:"keyspaces,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + RoutingRules *RoutingRules `protobuf:"bytes,2,opt,name=routing_rules,json=routingRules,proto3" json:"routing_rules,omitempty"` // table routing rules + ShardRoutingRules *ShardRoutingRules `protobuf:"bytes,3,opt,name=shard_routing_rules,json=shardRoutingRules,proto3" json:"shard_routing_rules,omitempty"` + KeyspaceRoutingRules *KeyspaceRoutingRules `protobuf:"bytes,4,opt,name=keyspace_routing_rules,json=keyspaceRoutingRules,proto3" json:"keyspace_routing_rules,omitempty"` } func (x *SrvVSchema) Reset() { *x = SrvVSchema{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[8] + mi := &file_vschema_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -735,7 +802,7 @@ func (x *SrvVSchema) String() string { func (*SrvVSchema) ProtoMessage() {} func (x *SrvVSchema) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[8] + mi := &file_vschema_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -748,7 +815,7 @@ func (x *SrvVSchema) ProtoReflect() protoreflect.Message { // Deprecated: Use SrvVSchema.ProtoReflect.Descriptor instead. func (*SrvVSchema) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{8} + return file_vschema_proto_rawDescGZIP(), []int{9} } func (x *SrvVSchema) GetKeyspaces() map[string]*Keyspace { @@ -772,6 +839,13 @@ func (x *SrvVSchema) GetShardRoutingRules() *ShardRoutingRules { return nil } +func (x *SrvVSchema) GetKeyspaceRoutingRules() *KeyspaceRoutingRules { + if x != nil { + return x.KeyspaceRoutingRules + } + return nil +} + // ShardRoutingRules specify the shard routing rules for the VSchema. type ShardRoutingRules struct { state protoimpl.MessageState @@ -784,7 +858,7 @@ type ShardRoutingRules struct { func (x *ShardRoutingRules) Reset() { *x = ShardRoutingRules{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[9] + mi := &file_vschema_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -797,7 +871,7 @@ func (x *ShardRoutingRules) String() string { func (*ShardRoutingRules) ProtoMessage() {} func (x *ShardRoutingRules) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[9] + mi := &file_vschema_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -810,7 +884,7 @@ func (x *ShardRoutingRules) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardRoutingRules.ProtoReflect.Descriptor instead. func (*ShardRoutingRules) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{9} + return file_vschema_proto_rawDescGZIP(), []int{10} } func (x *ShardRoutingRules) GetRules() []*ShardRoutingRule { @@ -820,7 +894,7 @@ func (x *ShardRoutingRules) GetRules() []*ShardRoutingRule { return nil } -// RoutingRule specifies a routing rule. +// ShardRoutingRule specifies a routing rule. type ShardRoutingRule struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -834,7 +908,7 @@ type ShardRoutingRule struct { func (x *ShardRoutingRule) Reset() { *x = ShardRoutingRule{} if protoimpl.UnsafeEnabled { - mi := &file_vschema_proto_msgTypes[10] + mi := &file_vschema_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -847,7 +921,7 @@ func (x *ShardRoutingRule) String() string { func (*ShardRoutingRule) ProtoMessage() {} func (x *ShardRoutingRule) ProtoReflect() protoreflect.Message { - mi := &file_vschema_proto_msgTypes[10] + mi := &file_vschema_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -860,7 +934,7 @@ func (x *ShardRoutingRule) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardRoutingRule.ProtoReflect.Descriptor instead. func (*ShardRoutingRule) Descriptor() ([]byte, []int) { - return file_vschema_proto_rawDescGZIP(), []int{10} + return file_vschema_proto_rawDescGZIP(), []int{11} } func (x *ShardRoutingRule) GetFromKeyspace() string { @@ -884,6 +958,108 @@ func (x *ShardRoutingRule) GetShard() string { return "" } +type KeyspaceRoutingRules struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Rules []*KeyspaceRoutingRule `protobuf:"bytes,1,rep,name=rules,proto3" json:"rules,omitempty"` +} + +func (x *KeyspaceRoutingRules) Reset() { + *x = KeyspaceRoutingRules{} + if protoimpl.UnsafeEnabled { + mi := &file_vschema_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *KeyspaceRoutingRules) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*KeyspaceRoutingRules) ProtoMessage() {} + +func (x *KeyspaceRoutingRules) ProtoReflect() protoreflect.Message { + mi := &file_vschema_proto_msgTypes[12] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use KeyspaceRoutingRules.ProtoReflect.Descriptor instead. +func (*KeyspaceRoutingRules) Descriptor() ([]byte, []int) { + return file_vschema_proto_rawDescGZIP(), []int{12} +} + +func (x *KeyspaceRoutingRules) GetRules() []*KeyspaceRoutingRule { + if x != nil { + return x.Rules + } + return nil +} + +type KeyspaceRoutingRule struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FromKeyspace string `protobuf:"bytes,1,opt,name=from_keyspace,json=fromKeyspace,proto3" json:"from_keyspace,omitempty"` + ToKeyspace string `protobuf:"bytes,2,opt,name=to_keyspace,json=toKeyspace,proto3" json:"to_keyspace,omitempty"` +} + +func (x *KeyspaceRoutingRule) Reset() { + *x = KeyspaceRoutingRule{} + if protoimpl.UnsafeEnabled { + mi := &file_vschema_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *KeyspaceRoutingRule) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*KeyspaceRoutingRule) ProtoMessage() {} + +func (x *KeyspaceRoutingRule) ProtoReflect() protoreflect.Message { + mi := &file_vschema_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use KeyspaceRoutingRule.ProtoReflect.Descriptor instead. +func (*KeyspaceRoutingRule) Descriptor() ([]byte, []int) { + return file_vschema_proto_rawDescGZIP(), []int{13} +} + +func (x *KeyspaceRoutingRule) GetFromKeyspace() string { + if x != nil { + return x.FromKeyspace + } + return "" +} + +func (x *KeyspaceRoutingRule) GetToKeyspace() string { + if x != nil { + return x.ToKeyspace + } + return "" +} + var File_vschema_proto protoreflect.FileDescriptor var file_vschema_proto_rawDesc = []byte{ @@ -897,7 +1073,7 @@ var file_vschema_proto_rawDesc = []byte{ 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x72, 0x6f, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0x84, 0x04, 0x0a, + 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0xca, 0x04, 0x0a, 0x08, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x68, 0x61, 0x72, 0x64, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x68, 0x61, 0x72, 0x64, 0x65, 0x64, 0x12, 0x3b, 0x0a, 0x08, 0x76, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x18, @@ -915,111 +1091,140 @@ var file_vschema_proto_rawDesc = []byte{ 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x2e, 0x46, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0e, 0x66, - 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x4c, 0x0a, - 0x0d, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 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, 0x25, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0f, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x49, 0x0a, 0x0b, 0x54, - 0x61, 0x62, 0x6c, 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, 0x24, 0x0a, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x76, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4b, 0x0a, 0x0e, 0x46, 0x6f, 0x72, 0x65, 0x69, 0x67, - 0x6e, 0x4b, 0x65, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x70, - 0x65, 0x63, 0x69, 0x66, 0x69, 0x65, 0x64, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x64, 0x69, 0x73, - 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x75, 0x6e, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x64, 0x10, 0x03, 0x22, 0xa2, 0x01, 0x0a, 0x06, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x12, - 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, - 0x70, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x56, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x1a, 0x39, 0x0a, - 0x0b, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb1, 0x02, 0x0a, 0x05, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x3e, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x5f, 0x76, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x56, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x12, 0x3d, 0x0a, 0x0e, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, - 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, - 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x63, - 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0d, 0x61, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x63, 0x72, - 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x12, 0x16, 0x0a, 0x06, 0x70, 0x69, 0x6e, 0x6e, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x70, 0x69, 0x6e, 0x6e, 0x65, 0x64, 0x12, 0x3a, 0x0a, 0x19, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, - 0x61, 0x74, 0x69, 0x76, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, 0x61, - 0x74, 0x69, 0x76, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0x54, 0x0a, 0x0c, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x16, 0x0a, 0x06, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x22, 0x43, 0x0a, 0x0d, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, - 0x65, 0x6e, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x73, - 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, - 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x22, 0x8c, 0x02, 0x0a, 0x06, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x69, 0x6e, 0x76, 0x69, 0x73, - 0x69, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x6e, 0x76, 0x69, - 0x73, 0x69, 0x62, 0x6c, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, - 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x63, - 0x61, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65, - 0x12, 0x1f, 0x0a, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x08, 0x20, 0x01, - 0x28, 0x08, 0x48, 0x00, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x88, 0x01, - 0x01, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x6e, 0x75, - 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x22, 0xa7, 0x02, 0x0a, 0x0a, 0x53, 0x72, 0x76, 0x56, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x09, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x2e, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x6b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x3a, 0x0a, 0x0d, 0x72, 0x6f, 0x75, 0x74, 0x69, - 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x02, 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, 0x4a, 0x0a, 0x13, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x72, 0x6f, 0x75, - 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x03, 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, 0x1a, - 0x4f, 0x0a, 0x0e, 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, 0x27, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x44, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, - 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 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, 0x52, - 0x05, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x22, 0x6e, 0x0a, 0x10, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x72, - 0x6f, 0x6d, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x66, 0x72, 0x6f, 0x6d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, - 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x4b, 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, 0x42, 0x26, 0x5a, 0x24, 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, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x44, 0x0a, + 0x11, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x73, 0x70, + 0x65, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x53, 0x70, + 0x65, 0x63, 0x52, 0x0f, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x53, + 0x70, 0x65, 0x63, 0x1a, 0x4c, 0x0a, 0x0d, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 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, 0x25, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, + 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x1a, 0x49, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 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, 0x24, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x0e, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x4b, 0x0a, 0x0e, + 0x46, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0f, + 0x0a, 0x0b, 0x75, 0x6e, 0x73, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x65, 0x64, 0x10, 0x00, 0x12, + 0x0c, 0x0a, 0x08, 0x64, 0x69, 0x73, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x10, 0x01, 0x12, 0x0d, 0x0a, + 0x09, 0x75, 0x6e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x64, 0x10, 0x03, 0x22, 0x84, 0x01, 0x0a, 0x0f, 0x4d, 0x75, + 0x6c, 0x74, 0x69, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x53, 0x70, 0x65, 0x63, 0x12, 0x31, 0x0a, + 0x15, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x74, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x3e, 0x0a, 0x15, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x5f, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x0b, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x12, 0x74, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, + 0x22, 0xa2, 0x01, 0x0a, 0x06, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x12, 0x0a, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, + 0x33, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x70, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x1a, 0x39, 0x0a, 0x0b, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb1, 0x02, 0x0a, 0x05, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, + 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x12, 0x3e, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x76, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x76, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x56, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x65, 0x73, 0x12, 0x3d, 0x0a, 0x0e, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x69, 0x6e, 0x63, 0x72, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x76, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x0d, 0x61, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x12, 0x29, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x16, 0x0a, + 0x06, 0x70, 0x69, 0x6e, 0x6e, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x70, + 0x69, 0x6e, 0x6e, 0x65, 0x64, 0x12, 0x3a, 0x0a, 0x19, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, + 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x61, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, 0x61, 0x74, 0x69, + 0x76, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x4c, 0x69, 0x73, 0x74, 0x41, 0x75, 0x74, 0x68, 0x6f, 0x72, 0x69, 0x74, 0x61, 0x74, 0x69, 0x76, + 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0x54, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, + 0x43, 0x0a, 0x0d, 0x41, 0x75, 0x74, 0x6f, 0x49, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x65, 0x22, 0x8c, 0x02, 0x0a, 0x06, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x0b, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x69, 0x6e, 0x76, 0x69, 0x73, 0x69, 0x62, 0x6c, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x69, 0x6e, 0x76, 0x69, 0x73, 0x69, 0x62, + 0x6c, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x25, 0x0a, 0x0e, + 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x1f, 0x0a, + 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x48, + 0x00, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x88, 0x01, 0x01, 0x12, 0x16, + 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x61, + 0x62, 0x6c, 0x65, 0x22, 0xfc, 0x02, 0x0a, 0x0a, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x09, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, + 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x6b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x12, 0x3a, 0x0a, 0x0d, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, + 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x02, 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, 0x4a, 0x0a, 0x13, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, + 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x03, 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, 0x53, 0x0a, 0x16, + 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, + 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x76, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x14, 0x6b, 0x65, 0x79, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, + 0x73, 0x1a, 0x4f, 0x0a, 0x0e, 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, 0x27, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x44, 0x0a, 0x11, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, + 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x2f, 0x0a, 0x05, 0x72, 0x75, 0x6c, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 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, 0x52, 0x05, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x22, 0x6e, 0x0a, 0x10, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, + 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x72, 0x6f, 0x6d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x4b, 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, 0x22, 0x4a, 0x0a, 0x14, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, + 0x12, 0x32, 0x0a, 0x05, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x05, 0x72, + 0x75, 0x6c, 0x65, 0x73, 0x22, 0x5b, 0x0a, 0x13, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x66, + 0x72, 0x6f, 0x6d, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0c, 0x66, 0x72, 0x6f, 0x6d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x42, 0x26, 0x5a, 0x24, 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, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, } var ( @@ -1035,48 +1240,55 @@ func file_vschema_proto_rawDescGZIP() []byte { } var file_vschema_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_vschema_proto_msgTypes = make([]protoimpl.MessageInfo, 15) +var file_vschema_proto_msgTypes = make([]protoimpl.MessageInfo, 18) var file_vschema_proto_goTypes = []interface{}{ (Keyspace_ForeignKeyMode)(0), // 0: vschema.Keyspace.ForeignKeyMode (*RoutingRules)(nil), // 1: vschema.RoutingRules (*RoutingRule)(nil), // 2: vschema.RoutingRule (*Keyspace)(nil), // 3: vschema.Keyspace - (*Vindex)(nil), // 4: vschema.Vindex - (*Table)(nil), // 5: vschema.Table - (*ColumnVindex)(nil), // 6: vschema.ColumnVindex - (*AutoIncrement)(nil), // 7: vschema.AutoIncrement - (*Column)(nil), // 8: vschema.Column - (*SrvVSchema)(nil), // 9: vschema.SrvVSchema - (*ShardRoutingRules)(nil), // 10: vschema.ShardRoutingRules - (*ShardRoutingRule)(nil), // 11: vschema.ShardRoutingRule - nil, // 12: vschema.Keyspace.VindexesEntry - nil, // 13: vschema.Keyspace.TablesEntry - nil, // 14: vschema.Vindex.ParamsEntry - nil, // 15: vschema.SrvVSchema.KeyspacesEntry - (query.Type)(0), // 16: query.Type + (*MultiTenantSpec)(nil), // 4: vschema.MultiTenantSpec + (*Vindex)(nil), // 5: vschema.Vindex + (*Table)(nil), // 6: vschema.Table + (*ColumnVindex)(nil), // 7: vschema.ColumnVindex + (*AutoIncrement)(nil), // 8: vschema.AutoIncrement + (*Column)(nil), // 9: vschema.Column + (*SrvVSchema)(nil), // 10: vschema.SrvVSchema + (*ShardRoutingRules)(nil), // 11: vschema.ShardRoutingRules + (*ShardRoutingRule)(nil), // 12: vschema.ShardRoutingRule + (*KeyspaceRoutingRules)(nil), // 13: vschema.KeyspaceRoutingRules + (*KeyspaceRoutingRule)(nil), // 14: vschema.KeyspaceRoutingRule + nil, // 15: vschema.Keyspace.VindexesEntry + nil, // 16: vschema.Keyspace.TablesEntry + nil, // 17: vschema.Vindex.ParamsEntry + nil, // 18: vschema.SrvVSchema.KeyspacesEntry + (query.Type)(0), // 19: query.Type } var file_vschema_proto_depIdxs = []int32{ 2, // 0: vschema.RoutingRules.rules:type_name -> vschema.RoutingRule - 12, // 1: vschema.Keyspace.vindexes:type_name -> vschema.Keyspace.VindexesEntry - 13, // 2: vschema.Keyspace.tables:type_name -> vschema.Keyspace.TablesEntry + 15, // 1: vschema.Keyspace.vindexes:type_name -> vschema.Keyspace.VindexesEntry + 16, // 2: vschema.Keyspace.tables:type_name -> vschema.Keyspace.TablesEntry 0, // 3: vschema.Keyspace.foreign_key_mode:type_name -> vschema.Keyspace.ForeignKeyMode - 14, // 4: vschema.Vindex.params:type_name -> vschema.Vindex.ParamsEntry - 6, // 5: vschema.Table.column_vindexes:type_name -> vschema.ColumnVindex - 7, // 6: vschema.Table.auto_increment:type_name -> vschema.AutoIncrement - 8, // 7: vschema.Table.columns:type_name -> vschema.Column - 16, // 8: vschema.Column.type:type_name -> query.Type - 15, // 9: vschema.SrvVSchema.keyspaces:type_name -> vschema.SrvVSchema.KeyspacesEntry - 1, // 10: vschema.SrvVSchema.routing_rules:type_name -> vschema.RoutingRules - 10, // 11: vschema.SrvVSchema.shard_routing_rules:type_name -> vschema.ShardRoutingRules - 11, // 12: vschema.ShardRoutingRules.rules:type_name -> vschema.ShardRoutingRule - 4, // 13: vschema.Keyspace.VindexesEntry.value:type_name -> vschema.Vindex - 5, // 14: vschema.Keyspace.TablesEntry.value:type_name -> vschema.Table - 3, // 15: vschema.SrvVSchema.KeyspacesEntry.value:type_name -> vschema.Keyspace - 16, // [16:16] is the sub-list for method output_type - 16, // [16:16] is the sub-list for method input_type - 16, // [16:16] is the sub-list for extension type_name - 16, // [16:16] is the sub-list for extension extendee - 0, // [0:16] is the sub-list for field type_name + 4, // 4: vschema.Keyspace.multi_tenant_spec:type_name -> vschema.MultiTenantSpec + 19, // 5: vschema.MultiTenantSpec.tenant_id_column_type:type_name -> query.Type + 17, // 6: vschema.Vindex.params:type_name -> vschema.Vindex.ParamsEntry + 7, // 7: vschema.Table.column_vindexes:type_name -> vschema.ColumnVindex + 8, // 8: vschema.Table.auto_increment:type_name -> vschema.AutoIncrement + 9, // 9: vschema.Table.columns:type_name -> vschema.Column + 19, // 10: vschema.Column.type:type_name -> query.Type + 18, // 11: vschema.SrvVSchema.keyspaces:type_name -> vschema.SrvVSchema.KeyspacesEntry + 1, // 12: vschema.SrvVSchema.routing_rules:type_name -> vschema.RoutingRules + 11, // 13: vschema.SrvVSchema.shard_routing_rules:type_name -> vschema.ShardRoutingRules + 13, // 14: vschema.SrvVSchema.keyspace_routing_rules:type_name -> vschema.KeyspaceRoutingRules + 12, // 15: vschema.ShardRoutingRules.rules:type_name -> vschema.ShardRoutingRule + 14, // 16: vschema.KeyspaceRoutingRules.rules:type_name -> vschema.KeyspaceRoutingRule + 5, // 17: vschema.Keyspace.VindexesEntry.value:type_name -> vschema.Vindex + 6, // 18: vschema.Keyspace.TablesEntry.value:type_name -> vschema.Table + 3, // 19: vschema.SrvVSchema.KeyspacesEntry.value:type_name -> vschema.Keyspace + 20, // [20:20] is the sub-list for method output_type + 20, // [20:20] is the sub-list for method input_type + 20, // [20:20] is the sub-list for extension type_name + 20, // [20:20] is the sub-list for extension extendee + 0, // [0:20] is the sub-list for field type_name } func init() { file_vschema_proto_init() } @@ -1122,7 +1334,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Vindex); i { + switch v := v.(*MultiTenantSpec); i { case 0: return &v.state case 1: @@ -1134,7 +1346,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Table); i { + switch v := v.(*Vindex); i { case 0: return &v.state case 1: @@ -1146,7 +1358,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ColumnVindex); i { + switch v := v.(*Table); i { case 0: return &v.state case 1: @@ -1158,7 +1370,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AutoIncrement); i { + switch v := v.(*ColumnVindex); i { case 0: return &v.state case 1: @@ -1170,7 +1382,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Column); i { + switch v := v.(*AutoIncrement); i { case 0: return &v.state case 1: @@ -1182,7 +1394,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SrvVSchema); i { + switch v := v.(*Column); i { case 0: return &v.state case 1: @@ -1194,7 +1406,7 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ShardRoutingRules); i { + switch v := v.(*SrvVSchema); i { case 0: return &v.state case 1: @@ -1206,6 +1418,18 @@ func file_vschema_proto_init() { } } file_vschema_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShardRoutingRules); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vschema_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardRoutingRule); i { case 0: return &v.state @@ -1217,15 +1441,39 @@ func file_vschema_proto_init() { return nil } } + file_vschema_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*KeyspaceRoutingRules); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vschema_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*KeyspaceRoutingRule); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } - file_vschema_proto_msgTypes[7].OneofWrappers = []interface{}{} + file_vschema_proto_msgTypes[8].OneofWrappers = []interface{}{} type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_vschema_proto_rawDesc, NumEnums: 1, - NumMessages: 15, + NumMessages: 18, NumExtensions: 0, NumServices: 0, }, diff --git a/go/vt/proto/vschema/vschema_vtproto.pb.go b/go/vt/proto/vschema/vschema_vtproto.pb.go index ba55bcfa97c..8cf523f4009 100644 --- a/go/vt/proto/vschema/vschema_vtproto.pb.go +++ b/go/vt/proto/vschema/vschema_vtproto.pb.go @@ -74,6 +74,7 @@ func (m *Keyspace) CloneVT() *Keyspace { Sharded: m.Sharded, RequireExplicitRouting: m.RequireExplicitRouting, ForeignKeyMode: m.ForeignKeyMode, + MultiTenantSpec: m.MultiTenantSpec.CloneVT(), } if rhs := m.Vindexes; rhs != nil { tmpContainer := make(map[string]*Vindex, len(rhs)) @@ -100,6 +101,25 @@ func (m *Keyspace) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *MultiTenantSpec) CloneVT() *MultiTenantSpec { + if m == nil { + return (*MultiTenantSpec)(nil) + } + r := &MultiTenantSpec{ + TenantIdColumnName: m.TenantIdColumnName, + TenantIdColumnType: m.TenantIdColumnType, + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *MultiTenantSpec) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *Vindex) CloneVT() *Vindex { if m == nil { return (*Vindex)(nil) @@ -243,8 +263,9 @@ func (m *SrvVSchema) CloneVT() *SrvVSchema { return (*SrvVSchema)(nil) } r := &SrvVSchema{ - RoutingRules: m.RoutingRules.CloneVT(), - ShardRoutingRules: m.ShardRoutingRules.CloneVT(), + RoutingRules: m.RoutingRules.CloneVT(), + ShardRoutingRules: m.ShardRoutingRules.CloneVT(), + KeyspaceRoutingRules: m.KeyspaceRoutingRules.CloneVT(), } if rhs := m.Keyspaces; rhs != nil { tmpContainer := make(map[string]*Keyspace, len(rhs)) @@ -307,6 +328,48 @@ func (m *ShardRoutingRule) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *KeyspaceRoutingRules) CloneVT() *KeyspaceRoutingRules { + if m == nil { + return (*KeyspaceRoutingRules)(nil) + } + r := &KeyspaceRoutingRules{} + if rhs := m.Rules; rhs != nil { + tmpContainer := make([]*KeyspaceRoutingRule, len(rhs)) + for k, v := range rhs { + tmpContainer[k] = v.CloneVT() + } + r.Rules = tmpContainer + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *KeyspaceRoutingRules) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *KeyspaceRoutingRule) CloneVT() *KeyspaceRoutingRule { + if m == nil { + return (*KeyspaceRoutingRule)(nil) + } + r := &KeyspaceRoutingRule{ + FromKeyspace: m.FromKeyspace, + ToKeyspace: m.ToKeyspace, + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *KeyspaceRoutingRule) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *RoutingRules) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -431,6 +494,16 @@ func (m *Keyspace) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.MultiTenantSpec != nil { + size, err := m.MultiTenantSpec.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } if m.ForeignKeyMode != 0 { i = encodeVarint(dAtA, i, uint64(m.ForeignKeyMode)) i-- @@ -503,6 +576,51 @@ func (m *Keyspace) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *MultiTenantSpec) 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 *MultiTenantSpec) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *MultiTenantSpec) 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.TenantIdColumnType != 0 { + i = encodeVarint(dAtA, i, uint64(m.TenantIdColumnType)) + i-- + dAtA[i] = 0x10 + } + if len(m.TenantIdColumnName) > 0 { + i -= len(m.TenantIdColumnName) + copy(dAtA[i:], m.TenantIdColumnName) + i = encodeVarint(dAtA, i, uint64(len(m.TenantIdColumnName))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *Vindex) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -898,6 +1016,16 @@ func (m *SrvVSchema) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.KeyspaceRoutingRules != nil { + size, err := m.KeyspaceRoutingRules.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x22 + } if m.ShardRoutingRules != nil { size, err := m.ShardRoutingRules.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { @@ -1042,6 +1170,98 @@ func (m *ShardRoutingRule) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *KeyspaceRoutingRules) 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 *KeyspaceRoutingRules) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *KeyspaceRoutingRules) 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 len(m.Rules) > 0 { + for iNdEx := len(m.Rules) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Rules[iNdEx].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 *KeyspaceRoutingRule) 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 *KeyspaceRoutingRule) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *KeyspaceRoutingRule) 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 len(m.ToKeyspace) > 0 { + i -= len(m.ToKeyspace) + copy(dAtA[i:], m.ToKeyspace) + i = encodeVarint(dAtA, i, uint64(len(m.ToKeyspace))) + i-- + dAtA[i] = 0x12 + } + if len(m.FromKeyspace) > 0 { + i -= len(m.FromKeyspace) + copy(dAtA[i:], m.FromKeyspace) + i = encodeVarint(dAtA, i, uint64(len(m.FromKeyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func encodeVarint(dAtA []byte, offset int, v uint64) int { offset -= sov(v) base := offset @@ -1130,6 +1350,27 @@ func (m *Keyspace) SizeVT() (n int) { if m.ForeignKeyMode != 0 { n += 1 + sov(uint64(m.ForeignKeyMode)) } + if m.MultiTenantSpec != nil { + l = m.MultiTenantSpec.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *MultiTenantSpec) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.TenantIdColumnName) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + if m.TenantIdColumnType != 0 { + n += 1 + sov(uint64(m.TenantIdColumnType)) + } n += len(m.unknownFields) return n } @@ -1313,6 +1554,10 @@ func (m *SrvVSchema) SizeVT() (n int) { l = m.ShardRoutingRules.SizeVT() n += 1 + l + sov(uint64(l)) } + if m.KeyspaceRoutingRules != nil { + l = m.KeyspaceRoutingRules.SizeVT() + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -1355,6 +1600,40 @@ func (m *ShardRoutingRule) SizeVT() (n int) { return n } +func (m *KeyspaceRoutingRules) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Rules) > 0 { + for _, e := range m.Rules { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *KeyspaceRoutingRule) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.FromKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.ToKeyspace) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + func sov(x uint64) (n int) { return (bits.Len64(x|1) + 6) / 7 } @@ -1907,13 +2186,151 @@ func (m *Keyspace) UnmarshalVT(dAtA []byte) error { break } } - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { - return err + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MultiTenantSpec", wireType) } - if (skippy < 0) || (iNdEx+skippy) < 0 { + 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.MultiTenantSpec == nil { + m.MultiTenantSpec = &MultiTenantSpec{} + } + if err := m.MultiTenantSpec.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 *MultiTenantSpec) 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: MultiTenantSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MultiTenantSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TenantIdColumnName", 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.TenantIdColumnName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TenantIdColumnType", wireType) + } + m.TenantIdColumnType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TenantIdColumnType |= query.Type(b&0x7F) << shift + if b < 0x80 { + break + } + } + 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 { @@ -3211,6 +3628,42 @@ func (m *SrvVSchema) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyspaceRoutingRules", 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.KeyspaceRoutingRules == nil { + m.KeyspaceRoutingRules = &KeyspaceRoutingRules{} + } + if err := m.KeyspaceRoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -3465,6 +3918,206 @@ func (m *ShardRoutingRule) UnmarshalVT(dAtA []byte) error { } return nil } +func (m *KeyspaceRoutingRules) 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: KeyspaceRoutingRules: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KeyspaceRoutingRules: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Rules", 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 + } + m.Rules = append(m.Rules, &KeyspaceRoutingRule{}) + if err := m.Rules[len(m.Rules)-1].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 *KeyspaceRoutingRule) 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: KeyspaceRoutingRule: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KeyspaceRoutingRule: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FromKeyspace", 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.FromKeyspace = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ToKeyspace", 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.ToKeyspace = string(dAtA[iNdEx:postIndex]) + 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 skip(dAtA []byte) (n int, err error) { l := len(dAtA) diff --git a/go/vt/proto/vtctldata/vtctldata.pb.go b/go/vt/proto/vtctldata/vtctldata.pb.go index 864135a23c8..cdfa274e47b 100644 --- a/go/vt/proto/vtctldata/vtctldata.pb.go +++ b/go/vt/proto/vtctldata/vtctldata.pb.go @@ -486,6 +486,7 @@ type MaterializeSettings struct { 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"` + WorkflowOptions *WorkflowOptions `protobuf:"bytes,17,opt,name=workflow_options,json=workflowOptions,proto3" json:"workflow_options,omitempty"` } func (x *MaterializeSettings) Reset() { @@ -632,6 +633,13 @@ func (x *MaterializeSettings) GetAtomicCopy() bool { return false } +func (x *MaterializeSettings) GetWorkflowOptions() *WorkflowOptions { + if x != nil { + return x.WorkflowOptions + } + return nil +} + type Keyspace struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1222,6 +1230,61 @@ func (x *Shard) GetShard() *topodata.Shard { return nil } +type WorkflowOptions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TenantId string `protobuf:"bytes,1,opt,name=tenant_id,json=tenantId,proto3" json:"tenant_id,omitempty"` + SourceKeyspaceAlias string `protobuf:"bytes,2,opt,name=source_keyspace_alias,json=sourceKeyspaceAlias,proto3" json:"source_keyspace_alias,omitempty"` +} + +func (x *WorkflowOptions) Reset() { + *x = WorkflowOptions{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowOptions) ProtoMessage() {} + +func (x *WorkflowOptions) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[7] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkflowOptions.ProtoReflect.Descriptor instead. +func (*WorkflowOptions) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{7} +} + +func (x *WorkflowOptions) GetTenantId() string { + if x != nil { + return x.TenantId + } + return "" +} + +func (x *WorkflowOptions) GetSourceKeyspaceAlias() string { + if x != nil { + return x.SourceKeyspaceAlias + } + return "" +} + // TODO: comment the hell out of this. type Workflow struct { state protoimpl.MessageState @@ -1243,12 +1306,15 @@ type Workflow struct { MaxVReplicationTransactionLag int64 `protobuf:"varint,8,opt,name=max_v_replication_transaction_lag,json=maxVReplicationTransactionLag,proto3" json:"max_v_replication_transaction_lag,omitempty"` // This specifies whether to defer the creation of secondary keys. DeferSecondaryKeys bool `protobuf:"varint,9,opt,name=defer_secondary_keys,json=deferSecondaryKeys,proto3" json:"defer_secondary_keys,omitempty"` + // These are additional (optional) settings for vreplication workflows. Previously we used to add it to the + // binlogdata.BinlogSource proto object. More details in go/vt/sidecardb/schema/vreplication.sql. + Options *WorkflowOptions `protobuf:"bytes,10,opt,name=options,proto3" json:"options,omitempty"` } func (x *Workflow) Reset() { *x = Workflow{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[7] + mi := &file_vtctldata_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1261,7 +1327,7 @@ func (x *Workflow) String() string { func (*Workflow) ProtoMessage() {} func (x *Workflow) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[7] + mi := &file_vtctldata_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1274,7 +1340,7 @@ func (x *Workflow) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow.ProtoReflect.Descriptor instead. func (*Workflow) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7} + return file_vtctldata_proto_rawDescGZIP(), []int{8} } func (x *Workflow) GetName() string { @@ -1340,6 +1406,13 @@ func (x *Workflow) GetDeferSecondaryKeys() bool { return false } +func (x *Workflow) GetOptions() *WorkflowOptions { + if x != nil { + return x.Options + } + return nil +} + type AddCellInfoRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1352,7 +1425,7 @@ type AddCellInfoRequest struct { func (x *AddCellInfoRequest) Reset() { *x = AddCellInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[8] + mi := &file_vtctldata_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1365,7 +1438,7 @@ func (x *AddCellInfoRequest) String() string { func (*AddCellInfoRequest) ProtoMessage() {} func (x *AddCellInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[8] + mi := &file_vtctldata_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1378,7 +1451,7 @@ func (x *AddCellInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AddCellInfoRequest.ProtoReflect.Descriptor instead. func (*AddCellInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{8} + return file_vtctldata_proto_rawDescGZIP(), []int{9} } func (x *AddCellInfoRequest) GetName() string { @@ -1404,7 +1477,7 @@ type AddCellInfoResponse struct { func (x *AddCellInfoResponse) Reset() { *x = AddCellInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[9] + mi := &file_vtctldata_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1417,7 +1490,7 @@ func (x *AddCellInfoResponse) String() string { func (*AddCellInfoResponse) ProtoMessage() {} func (x *AddCellInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[9] + mi := &file_vtctldata_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1430,7 +1503,7 @@ func (x *AddCellInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AddCellInfoResponse.ProtoReflect.Descriptor instead. func (*AddCellInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{9} + return file_vtctldata_proto_rawDescGZIP(), []int{10} } type AddCellsAliasRequest struct { @@ -1445,7 +1518,7 @@ type AddCellsAliasRequest struct { func (x *AddCellsAliasRequest) Reset() { *x = AddCellsAliasRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[10] + mi := &file_vtctldata_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1458,7 +1531,7 @@ func (x *AddCellsAliasRequest) String() string { func (*AddCellsAliasRequest) ProtoMessage() {} func (x *AddCellsAliasRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[10] + mi := &file_vtctldata_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1471,7 +1544,7 @@ func (x *AddCellsAliasRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AddCellsAliasRequest.ProtoReflect.Descriptor instead. func (*AddCellsAliasRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{10} + return file_vtctldata_proto_rawDescGZIP(), []int{11} } func (x *AddCellsAliasRequest) GetName() string { @@ -1497,7 +1570,7 @@ type AddCellsAliasResponse struct { func (x *AddCellsAliasResponse) Reset() { *x = AddCellsAliasResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[11] + mi := &file_vtctldata_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1510,7 +1583,7 @@ func (x *AddCellsAliasResponse) String() string { func (*AddCellsAliasResponse) ProtoMessage() {} func (x *AddCellsAliasResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[11] + mi := &file_vtctldata_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1523,7 +1596,114 @@ func (x *AddCellsAliasResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AddCellsAliasResponse.ProtoReflect.Descriptor instead. func (*AddCellsAliasResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{11} + return file_vtctldata_proto_rawDescGZIP(), []int{12} +} + +type ApplyKeyspaceRoutingRulesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + KeyspaceRoutingRules *vschema.KeyspaceRoutingRules `protobuf:"bytes,1,opt,name=keyspace_routing_rules,json=keyspaceRoutingRules,proto3" json:"keyspace_routing_rules,omitempty"` + // SkipRebuild, if set, will cause ApplyKeyspaceRoutingRules to skip rebuilding the + // SrvVSchema objects in each cell in RebuildCells. + SkipRebuild bool `protobuf:"varint,2,opt,name=skip_rebuild,json=skipRebuild,proto3" json:"skip_rebuild,omitempty"` + // RebuildCells limits the SrvVSchema rebuild to the specified cells. If not + // provided the SrvVSchema will be rebuilt in every cell in the topology. + // + // Ignored if SkipRebuild is set. + RebuildCells []string `protobuf:"bytes,3,rep,name=rebuild_cells,json=rebuildCells,proto3" json:"rebuild_cells,omitempty"` +} + +func (x *ApplyKeyspaceRoutingRulesRequest) Reset() { + *x = ApplyKeyspaceRoutingRulesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ApplyKeyspaceRoutingRulesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApplyKeyspaceRoutingRulesRequest) ProtoMessage() {} + +func (x *ApplyKeyspaceRoutingRulesRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ApplyKeyspaceRoutingRulesRequest.ProtoReflect.Descriptor instead. +func (*ApplyKeyspaceRoutingRulesRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{13} +} + +func (x *ApplyKeyspaceRoutingRulesRequest) GetKeyspaceRoutingRules() *vschema.KeyspaceRoutingRules { + if x != nil { + return x.KeyspaceRoutingRules + } + return nil +} + +func (x *ApplyKeyspaceRoutingRulesRequest) GetSkipRebuild() bool { + if x != nil { + return x.SkipRebuild + } + return false +} + +func (x *ApplyKeyspaceRoutingRulesRequest) GetRebuildCells() []string { + if x != nil { + return x.RebuildCells + } + return nil +} + +type ApplyKeyspaceRoutingRulesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ApplyKeyspaceRoutingRulesResponse) Reset() { + *x = ApplyKeyspaceRoutingRulesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ApplyKeyspaceRoutingRulesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApplyKeyspaceRoutingRulesResponse) ProtoMessage() {} + +func (x *ApplyKeyspaceRoutingRulesResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ApplyKeyspaceRoutingRulesResponse.ProtoReflect.Descriptor instead. +func (*ApplyKeyspaceRoutingRulesResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{14} } type ApplyRoutingRulesRequest struct { @@ -1545,7 +1725,7 @@ type ApplyRoutingRulesRequest struct { func (x *ApplyRoutingRulesRequest) Reset() { *x = ApplyRoutingRulesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[12] + mi := &file_vtctldata_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1558,7 +1738,7 @@ func (x *ApplyRoutingRulesRequest) String() string { func (*ApplyRoutingRulesRequest) ProtoMessage() {} func (x *ApplyRoutingRulesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[12] + mi := &file_vtctldata_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1571,7 +1751,7 @@ func (x *ApplyRoutingRulesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyRoutingRulesRequest.ProtoReflect.Descriptor instead. func (*ApplyRoutingRulesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{12} + return file_vtctldata_proto_rawDescGZIP(), []int{15} } func (x *ApplyRoutingRulesRequest) GetRoutingRules() *vschema.RoutingRules { @@ -1604,7 +1784,7 @@ type ApplyRoutingRulesResponse struct { func (x *ApplyRoutingRulesResponse) Reset() { *x = ApplyRoutingRulesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[13] + mi := &file_vtctldata_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1617,7 +1797,7 @@ func (x *ApplyRoutingRulesResponse) String() string { func (*ApplyRoutingRulesResponse) ProtoMessage() {} func (x *ApplyRoutingRulesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[13] + mi := &file_vtctldata_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1630,7 +1810,7 @@ func (x *ApplyRoutingRulesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyRoutingRulesResponse.ProtoReflect.Descriptor instead. func (*ApplyRoutingRulesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{13} + return file_vtctldata_proto_rawDescGZIP(), []int{16} } type ApplyShardRoutingRulesRequest struct { @@ -1652,7 +1832,7 @@ type ApplyShardRoutingRulesRequest struct { func (x *ApplyShardRoutingRulesRequest) Reset() { *x = ApplyShardRoutingRulesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[14] + mi := &file_vtctldata_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1665,7 +1845,7 @@ func (x *ApplyShardRoutingRulesRequest) String() string { func (*ApplyShardRoutingRulesRequest) ProtoMessage() {} func (x *ApplyShardRoutingRulesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[14] + mi := &file_vtctldata_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1678,7 +1858,7 @@ func (x *ApplyShardRoutingRulesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyShardRoutingRulesRequest.ProtoReflect.Descriptor instead. func (*ApplyShardRoutingRulesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{14} + return file_vtctldata_proto_rawDescGZIP(), []int{17} } func (x *ApplyShardRoutingRulesRequest) GetShardRoutingRules() *vschema.ShardRoutingRules { @@ -1711,7 +1891,7 @@ type ApplyShardRoutingRulesResponse struct { func (x *ApplyShardRoutingRulesResponse) Reset() { *x = ApplyShardRoutingRulesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[15] + mi := &file_vtctldata_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1724,7 +1904,7 @@ func (x *ApplyShardRoutingRulesResponse) String() string { func (*ApplyShardRoutingRulesResponse) ProtoMessage() {} func (x *ApplyShardRoutingRulesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[15] + mi := &file_vtctldata_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1737,7 +1917,7 @@ func (x *ApplyShardRoutingRulesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyShardRoutingRulesResponse.ProtoReflect.Descriptor instead. func (*ApplyShardRoutingRulesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{15} + return file_vtctldata_proto_rawDescGZIP(), []int{18} } type ApplySchemaRequest struct { @@ -1769,7 +1949,7 @@ type ApplySchemaRequest struct { func (x *ApplySchemaRequest) Reset() { *x = ApplySchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[16] + mi := &file_vtctldata_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1782,7 +1962,7 @@ func (x *ApplySchemaRequest) String() string { func (*ApplySchemaRequest) ProtoMessage() {} func (x *ApplySchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[16] + mi := &file_vtctldata_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1795,7 +1975,7 @@ func (x *ApplySchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplySchemaRequest.ProtoReflect.Descriptor instead. func (*ApplySchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{16} + return file_vtctldata_proto_rawDescGZIP(), []int{19} } func (x *ApplySchemaRequest) GetKeyspace() string { @@ -1866,7 +2046,7 @@ type ApplySchemaResponse struct { func (x *ApplySchemaResponse) Reset() { *x = ApplySchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[17] + mi := &file_vtctldata_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1879,7 +2059,7 @@ func (x *ApplySchemaResponse) String() string { func (*ApplySchemaResponse) ProtoMessage() {} func (x *ApplySchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[17] + mi := &file_vtctldata_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1892,7 +2072,7 @@ func (x *ApplySchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplySchemaResponse.ProtoReflect.Descriptor instead. func (*ApplySchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{17} + return file_vtctldata_proto_rawDescGZIP(), []int{20} } func (x *ApplySchemaResponse) GetUuidList() []string { @@ -1927,7 +2107,7 @@ type ApplyVSchemaRequest struct { func (x *ApplyVSchemaRequest) Reset() { *x = ApplyVSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[18] + mi := &file_vtctldata_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1940,7 +2120,7 @@ func (x *ApplyVSchemaRequest) String() string { func (*ApplyVSchemaRequest) ProtoMessage() {} func (x *ApplyVSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[18] + mi := &file_vtctldata_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1953,7 +2133,7 @@ func (x *ApplyVSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyVSchemaRequest.ProtoReflect.Descriptor instead. func (*ApplyVSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{18} + return file_vtctldata_proto_rawDescGZIP(), []int{21} } func (x *ApplyVSchemaRequest) GetKeyspace() string { @@ -2025,7 +2205,7 @@ type ApplyVSchemaResponse struct { func (x *ApplyVSchemaResponse) Reset() { *x = ApplyVSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[19] + mi := &file_vtctldata_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2038,7 +2218,7 @@ func (x *ApplyVSchemaResponse) String() string { func (*ApplyVSchemaResponse) ProtoMessage() {} func (x *ApplyVSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[19] + mi := &file_vtctldata_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2051,7 +2231,7 @@ func (x *ApplyVSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyVSchemaResponse.ProtoReflect.Descriptor instead. func (*ApplyVSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{19} + return file_vtctldata_proto_rawDescGZIP(), []int{22} } func (x *ApplyVSchemaResponse) GetVSchema() *vschema.Keyspace { @@ -2093,7 +2273,7 @@ type BackupRequest struct { func (x *BackupRequest) Reset() { *x = BackupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[20] + mi := &file_vtctldata_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2106,7 +2286,7 @@ func (x *BackupRequest) String() string { func (*BackupRequest) ProtoMessage() {} func (x *BackupRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[20] + mi := &file_vtctldata_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2119,7 +2299,7 @@ func (x *BackupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupRequest.ProtoReflect.Descriptor instead. func (*BackupRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{20} + return file_vtctldata_proto_rawDescGZIP(), []int{23} } func (x *BackupRequest) GetTabletAlias() *topodata.TabletAlias { @@ -2172,7 +2352,7 @@ type BackupResponse struct { func (x *BackupResponse) Reset() { *x = BackupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[21] + mi := &file_vtctldata_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2185,7 +2365,7 @@ func (x *BackupResponse) String() string { func (*BackupResponse) ProtoMessage() {} func (x *BackupResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[21] + mi := &file_vtctldata_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2198,7 +2378,7 @@ func (x *BackupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupResponse.ProtoReflect.Descriptor instead. func (*BackupResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{21} + return file_vtctldata_proto_rawDescGZIP(), []int{24} } func (x *BackupResponse) GetTabletAlias() *topodata.TabletAlias { @@ -2253,7 +2433,7 @@ type BackupShardRequest struct { func (x *BackupShardRequest) Reset() { *x = BackupShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[22] + mi := &file_vtctldata_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2266,7 +2446,7 @@ func (x *BackupShardRequest) String() string { func (*BackupShardRequest) ProtoMessage() {} func (x *BackupShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[22] + mi := &file_vtctldata_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2279,7 +2459,7 @@ func (x *BackupShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BackupShardRequest.ProtoReflect.Descriptor instead. func (*BackupShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{22} + return file_vtctldata_proto_rawDescGZIP(), []int{25} } func (x *BackupShardRequest) GetKeyspace() string { @@ -2336,7 +2516,7 @@ type CancelSchemaMigrationRequest struct { func (x *CancelSchemaMigrationRequest) Reset() { *x = CancelSchemaMigrationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[23] + mi := &file_vtctldata_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2349,7 +2529,7 @@ func (x *CancelSchemaMigrationRequest) String() string { func (*CancelSchemaMigrationRequest) ProtoMessage() {} func (x *CancelSchemaMigrationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[23] + mi := &file_vtctldata_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2362,7 +2542,7 @@ func (x *CancelSchemaMigrationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CancelSchemaMigrationRequest.ProtoReflect.Descriptor instead. func (*CancelSchemaMigrationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{23} + return file_vtctldata_proto_rawDescGZIP(), []int{26} } func (x *CancelSchemaMigrationRequest) GetKeyspace() string { @@ -2390,7 +2570,7 @@ type CancelSchemaMigrationResponse struct { func (x *CancelSchemaMigrationResponse) Reset() { *x = CancelSchemaMigrationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[24] + mi := &file_vtctldata_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2403,7 +2583,7 @@ func (x *CancelSchemaMigrationResponse) String() string { func (*CancelSchemaMigrationResponse) ProtoMessage() {} func (x *CancelSchemaMigrationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[24] + mi := &file_vtctldata_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2416,7 +2596,7 @@ func (x *CancelSchemaMigrationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CancelSchemaMigrationResponse.ProtoReflect.Descriptor instead. func (*CancelSchemaMigrationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{24} + return file_vtctldata_proto_rawDescGZIP(), []int{27} } func (x *CancelSchemaMigrationResponse) GetRowsAffectedByShard() map[string]uint64 { @@ -2439,7 +2619,7 @@ type ChangeTabletTypeRequest struct { func (x *ChangeTabletTypeRequest) Reset() { *x = ChangeTabletTypeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[25] + mi := &file_vtctldata_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2452,7 +2632,7 @@ func (x *ChangeTabletTypeRequest) String() string { func (*ChangeTabletTypeRequest) ProtoMessage() {} func (x *ChangeTabletTypeRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[25] + mi := &file_vtctldata_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2465,7 +2645,7 @@ func (x *ChangeTabletTypeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ChangeTabletTypeRequest.ProtoReflect.Descriptor instead. func (*ChangeTabletTypeRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{25} + return file_vtctldata_proto_rawDescGZIP(), []int{28} } func (x *ChangeTabletTypeRequest) GetTabletAlias() *topodata.TabletAlias { @@ -2502,7 +2682,7 @@ type ChangeTabletTypeResponse struct { func (x *ChangeTabletTypeResponse) Reset() { *x = ChangeTabletTypeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[26] + mi := &file_vtctldata_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2515,7 +2695,7 @@ func (x *ChangeTabletTypeResponse) String() string { func (*ChangeTabletTypeResponse) ProtoMessage() {} func (x *ChangeTabletTypeResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[26] + mi := &file_vtctldata_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2528,7 +2708,7 @@ func (x *ChangeTabletTypeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ChangeTabletTypeResponse.ProtoReflect.Descriptor instead. func (*ChangeTabletTypeResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{26} + return file_vtctldata_proto_rawDescGZIP(), []int{29} } func (x *ChangeTabletTypeResponse) GetBeforeTablet() *topodata.Tablet { @@ -2564,7 +2744,7 @@ type CleanupSchemaMigrationRequest struct { func (x *CleanupSchemaMigrationRequest) Reset() { *x = CleanupSchemaMigrationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[27] + mi := &file_vtctldata_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2577,7 +2757,7 @@ func (x *CleanupSchemaMigrationRequest) String() string { func (*CleanupSchemaMigrationRequest) ProtoMessage() {} func (x *CleanupSchemaMigrationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[27] + mi := &file_vtctldata_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2590,7 +2770,7 @@ func (x *CleanupSchemaMigrationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CleanupSchemaMigrationRequest.ProtoReflect.Descriptor instead. func (*CleanupSchemaMigrationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{27} + return file_vtctldata_proto_rawDescGZIP(), []int{30} } func (x *CleanupSchemaMigrationRequest) GetKeyspace() string { @@ -2618,7 +2798,7 @@ type CleanupSchemaMigrationResponse struct { func (x *CleanupSchemaMigrationResponse) Reset() { *x = CleanupSchemaMigrationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[28] + mi := &file_vtctldata_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2631,7 +2811,7 @@ func (x *CleanupSchemaMigrationResponse) String() string { func (*CleanupSchemaMigrationResponse) ProtoMessage() {} func (x *CleanupSchemaMigrationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[28] + mi := &file_vtctldata_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2644,7 +2824,7 @@ func (x *CleanupSchemaMigrationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CleanupSchemaMigrationResponse.ProtoReflect.Descriptor instead. func (*CleanupSchemaMigrationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{28} + return file_vtctldata_proto_rawDescGZIP(), []int{31} } func (x *CleanupSchemaMigrationResponse) GetRowsAffectedByShard() map[string]uint64 { @@ -2666,7 +2846,7 @@ type CompleteSchemaMigrationRequest struct { func (x *CompleteSchemaMigrationRequest) Reset() { *x = CompleteSchemaMigrationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[29] + mi := &file_vtctldata_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2679,7 +2859,7 @@ func (x *CompleteSchemaMigrationRequest) String() string { func (*CompleteSchemaMigrationRequest) ProtoMessage() {} func (x *CompleteSchemaMigrationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[29] + mi := &file_vtctldata_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2692,7 +2872,7 @@ func (x *CompleteSchemaMigrationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CompleteSchemaMigrationRequest.ProtoReflect.Descriptor instead. func (*CompleteSchemaMigrationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{29} + return file_vtctldata_proto_rawDescGZIP(), []int{32} } func (x *CompleteSchemaMigrationRequest) GetKeyspace() string { @@ -2720,7 +2900,7 @@ type CompleteSchemaMigrationResponse struct { func (x *CompleteSchemaMigrationResponse) Reset() { *x = CompleteSchemaMigrationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[30] + mi := &file_vtctldata_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2733,7 +2913,7 @@ func (x *CompleteSchemaMigrationResponse) String() string { func (*CompleteSchemaMigrationResponse) ProtoMessage() {} func (x *CompleteSchemaMigrationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[30] + mi := &file_vtctldata_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2746,7 +2926,7 @@ func (x *CompleteSchemaMigrationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CompleteSchemaMigrationResponse.ProtoReflect.Descriptor instead. func (*CompleteSchemaMigrationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{30} + return file_vtctldata_proto_rawDescGZIP(), []int{33} } func (x *CompleteSchemaMigrationResponse) GetRowsAffectedByShard() map[string]uint64 { @@ -2786,7 +2966,7 @@ type CreateKeyspaceRequest struct { func (x *CreateKeyspaceRequest) Reset() { *x = CreateKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[31] + mi := &file_vtctldata_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2799,7 +2979,7 @@ func (x *CreateKeyspaceRequest) String() string { func (*CreateKeyspaceRequest) ProtoMessage() {} func (x *CreateKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[31] + mi := &file_vtctldata_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2812,7 +2992,7 @@ func (x *CreateKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateKeyspaceRequest.ProtoReflect.Descriptor instead. func (*CreateKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{31} + return file_vtctldata_proto_rawDescGZIP(), []int{34} } func (x *CreateKeyspaceRequest) GetName() string { @@ -2883,7 +3063,7 @@ type CreateKeyspaceResponse struct { func (x *CreateKeyspaceResponse) Reset() { *x = CreateKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[32] + mi := &file_vtctldata_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2896,7 +3076,7 @@ func (x *CreateKeyspaceResponse) String() string { func (*CreateKeyspaceResponse) ProtoMessage() {} func (x *CreateKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[32] + mi := &file_vtctldata_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2909,7 +3089,7 @@ func (x *CreateKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateKeyspaceResponse.ProtoReflect.Descriptor instead. func (*CreateKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{32} + return file_vtctldata_proto_rawDescGZIP(), []int{35} } func (x *CreateKeyspaceResponse) GetKeyspace() *Keyspace { @@ -2939,7 +3119,7 @@ type CreateShardRequest struct { func (x *CreateShardRequest) Reset() { *x = CreateShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[33] + mi := &file_vtctldata_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2952,7 +3132,7 @@ func (x *CreateShardRequest) String() string { func (*CreateShardRequest) ProtoMessage() {} func (x *CreateShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[33] + mi := &file_vtctldata_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2965,7 +3145,7 @@ func (x *CreateShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateShardRequest.ProtoReflect.Descriptor instead. func (*CreateShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{33} + return file_vtctldata_proto_rawDescGZIP(), []int{36} } func (x *CreateShardRequest) GetKeyspace() string { @@ -3014,7 +3194,7 @@ type CreateShardResponse struct { func (x *CreateShardResponse) Reset() { *x = CreateShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[34] + mi := &file_vtctldata_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3027,7 +3207,7 @@ func (x *CreateShardResponse) String() string { func (*CreateShardResponse) ProtoMessage() {} func (x *CreateShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[34] + mi := &file_vtctldata_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3040,7 +3220,7 @@ func (x *CreateShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateShardResponse.ProtoReflect.Descriptor instead. func (*CreateShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{34} + return file_vtctldata_proto_rawDescGZIP(), []int{37} } func (x *CreateShardResponse) GetKeyspace() *Keyspace { @@ -3076,7 +3256,7 @@ type DeleteCellInfoRequest struct { func (x *DeleteCellInfoRequest) Reset() { *x = DeleteCellInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[35] + mi := &file_vtctldata_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3089,7 +3269,7 @@ func (x *DeleteCellInfoRequest) String() string { func (*DeleteCellInfoRequest) ProtoMessage() {} func (x *DeleteCellInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[35] + mi := &file_vtctldata_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3102,7 +3282,7 @@ func (x *DeleteCellInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteCellInfoRequest.ProtoReflect.Descriptor instead. func (*DeleteCellInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{35} + return file_vtctldata_proto_rawDescGZIP(), []int{38} } func (x *DeleteCellInfoRequest) GetName() string { @@ -3128,7 +3308,7 @@ type DeleteCellInfoResponse struct { func (x *DeleteCellInfoResponse) Reset() { *x = DeleteCellInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[36] + mi := &file_vtctldata_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3141,7 +3321,7 @@ func (x *DeleteCellInfoResponse) String() string { func (*DeleteCellInfoResponse) ProtoMessage() {} func (x *DeleteCellInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[36] + mi := &file_vtctldata_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3154,7 +3334,7 @@ func (x *DeleteCellInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteCellInfoResponse.ProtoReflect.Descriptor instead. func (*DeleteCellInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{36} + return file_vtctldata_proto_rawDescGZIP(), []int{39} } type DeleteCellsAliasRequest struct { @@ -3168,7 +3348,7 @@ type DeleteCellsAliasRequest struct { func (x *DeleteCellsAliasRequest) Reset() { *x = DeleteCellsAliasRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[37] + mi := &file_vtctldata_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3181,7 +3361,7 @@ func (x *DeleteCellsAliasRequest) String() string { func (*DeleteCellsAliasRequest) ProtoMessage() {} func (x *DeleteCellsAliasRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[37] + mi := &file_vtctldata_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3194,7 +3374,7 @@ func (x *DeleteCellsAliasRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteCellsAliasRequest.ProtoReflect.Descriptor instead. func (*DeleteCellsAliasRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{37} + return file_vtctldata_proto_rawDescGZIP(), []int{40} } func (x *DeleteCellsAliasRequest) GetName() string { @@ -3213,7 +3393,7 @@ type DeleteCellsAliasResponse struct { func (x *DeleteCellsAliasResponse) Reset() { *x = DeleteCellsAliasResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[38] + mi := &file_vtctldata_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3226,7 +3406,7 @@ func (x *DeleteCellsAliasResponse) String() string { func (*DeleteCellsAliasResponse) ProtoMessage() {} func (x *DeleteCellsAliasResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[38] + mi := &file_vtctldata_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3239,7 +3419,7 @@ func (x *DeleteCellsAliasResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteCellsAliasResponse.ProtoReflect.Descriptor instead. func (*DeleteCellsAliasResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{38} + return file_vtctldata_proto_rawDescGZIP(), []int{41} } type DeleteKeyspaceRequest struct { @@ -3261,7 +3441,7 @@ type DeleteKeyspaceRequest struct { func (x *DeleteKeyspaceRequest) Reset() { *x = DeleteKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[39] + mi := &file_vtctldata_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3274,7 +3454,7 @@ func (x *DeleteKeyspaceRequest) String() string { func (*DeleteKeyspaceRequest) ProtoMessage() {} func (x *DeleteKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[39] + mi := &file_vtctldata_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3287,7 +3467,7 @@ func (x *DeleteKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteKeyspaceRequest.ProtoReflect.Descriptor instead. func (*DeleteKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{39} + return file_vtctldata_proto_rawDescGZIP(), []int{42} } func (x *DeleteKeyspaceRequest) GetKeyspace() string { @@ -3320,7 +3500,7 @@ type DeleteKeyspaceResponse struct { func (x *DeleteKeyspaceResponse) Reset() { *x = DeleteKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[40] + mi := &file_vtctldata_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3333,7 +3513,7 @@ func (x *DeleteKeyspaceResponse) String() string { func (*DeleteKeyspaceResponse) ProtoMessage() {} func (x *DeleteKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[40] + mi := &file_vtctldata_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3346,7 +3526,7 @@ func (x *DeleteKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteKeyspaceResponse.ProtoReflect.Descriptor instead. func (*DeleteKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{40} + return file_vtctldata_proto_rawDescGZIP(), []int{43} } type DeleteShardsRequest struct { @@ -3372,7 +3552,7 @@ type DeleteShardsRequest struct { func (x *DeleteShardsRequest) Reset() { *x = DeleteShardsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[41] + mi := &file_vtctldata_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3385,7 +3565,7 @@ func (x *DeleteShardsRequest) String() string { func (*DeleteShardsRequest) ProtoMessage() {} func (x *DeleteShardsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[41] + mi := &file_vtctldata_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3398,7 +3578,7 @@ func (x *DeleteShardsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteShardsRequest.ProtoReflect.Descriptor instead. func (*DeleteShardsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{41} + return file_vtctldata_proto_rawDescGZIP(), []int{44} } func (x *DeleteShardsRequest) GetShards() []*Shard { @@ -3438,7 +3618,7 @@ type DeleteShardsResponse struct { func (x *DeleteShardsResponse) Reset() { *x = DeleteShardsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[42] + mi := &file_vtctldata_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3451,7 +3631,7 @@ func (x *DeleteShardsResponse) String() string { func (*DeleteShardsResponse) ProtoMessage() {} func (x *DeleteShardsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[42] + mi := &file_vtctldata_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3464,7 +3644,7 @@ func (x *DeleteShardsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteShardsResponse.ProtoReflect.Descriptor instead. func (*DeleteShardsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{42} + return file_vtctldata_proto_rawDescGZIP(), []int{45} } type DeleteSrvVSchemaRequest struct { @@ -3478,7 +3658,7 @@ type DeleteSrvVSchemaRequest struct { func (x *DeleteSrvVSchemaRequest) Reset() { *x = DeleteSrvVSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[43] + mi := &file_vtctldata_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3491,7 +3671,7 @@ func (x *DeleteSrvVSchemaRequest) String() string { func (*DeleteSrvVSchemaRequest) ProtoMessage() {} func (x *DeleteSrvVSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[43] + mi := &file_vtctldata_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3504,7 +3684,7 @@ func (x *DeleteSrvVSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteSrvVSchemaRequest.ProtoReflect.Descriptor instead. func (*DeleteSrvVSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{43} + return file_vtctldata_proto_rawDescGZIP(), []int{46} } func (x *DeleteSrvVSchemaRequest) GetCell() string { @@ -3523,7 +3703,7 @@ type DeleteSrvVSchemaResponse struct { func (x *DeleteSrvVSchemaResponse) Reset() { *x = DeleteSrvVSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[44] + mi := &file_vtctldata_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3536,7 +3716,7 @@ func (x *DeleteSrvVSchemaResponse) String() string { func (*DeleteSrvVSchemaResponse) ProtoMessage() {} func (x *DeleteSrvVSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[44] + mi := &file_vtctldata_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3549,7 +3729,7 @@ func (x *DeleteSrvVSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteSrvVSchemaResponse.ProtoReflect.Descriptor instead. func (*DeleteSrvVSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{44} + return file_vtctldata_proto_rawDescGZIP(), []int{47} } type DeleteTabletsRequest struct { @@ -3567,7 +3747,7 @@ type DeleteTabletsRequest struct { func (x *DeleteTabletsRequest) Reset() { *x = DeleteTabletsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[45] + mi := &file_vtctldata_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3580,7 +3760,7 @@ func (x *DeleteTabletsRequest) String() string { func (*DeleteTabletsRequest) ProtoMessage() {} func (x *DeleteTabletsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[45] + mi := &file_vtctldata_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3593,7 +3773,7 @@ func (x *DeleteTabletsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteTabletsRequest.ProtoReflect.Descriptor instead. func (*DeleteTabletsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{45} + return file_vtctldata_proto_rawDescGZIP(), []int{48} } func (x *DeleteTabletsRequest) GetTabletAliases() []*topodata.TabletAlias { @@ -3619,7 +3799,7 @@ type DeleteTabletsResponse struct { func (x *DeleteTabletsResponse) Reset() { *x = DeleteTabletsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[46] + mi := &file_vtctldata_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3632,7 +3812,7 @@ func (x *DeleteTabletsResponse) String() string { func (*DeleteTabletsResponse) ProtoMessage() {} func (x *DeleteTabletsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[46] + mi := &file_vtctldata_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3645,7 +3825,7 @@ func (x *DeleteTabletsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteTabletsResponse.ProtoReflect.Descriptor instead. func (*DeleteTabletsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{46} + return file_vtctldata_proto_rawDescGZIP(), []int{49} } type EmergencyReparentShardRequest struct { @@ -3679,7 +3859,7 @@ type EmergencyReparentShardRequest struct { func (x *EmergencyReparentShardRequest) Reset() { *x = EmergencyReparentShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[47] + mi := &file_vtctldata_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3692,7 +3872,7 @@ func (x *EmergencyReparentShardRequest) String() string { func (*EmergencyReparentShardRequest) ProtoMessage() {} func (x *EmergencyReparentShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[47] + mi := &file_vtctldata_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3705,7 +3885,7 @@ func (x *EmergencyReparentShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use EmergencyReparentShardRequest.ProtoReflect.Descriptor instead. func (*EmergencyReparentShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{47} + return file_vtctldata_proto_rawDescGZIP(), []int{50} } func (x *EmergencyReparentShardRequest) GetKeyspace() string { @@ -3777,7 +3957,7 @@ type EmergencyReparentShardResponse struct { func (x *EmergencyReparentShardResponse) Reset() { *x = EmergencyReparentShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[48] + mi := &file_vtctldata_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3790,7 +3970,7 @@ func (x *EmergencyReparentShardResponse) String() string { func (*EmergencyReparentShardResponse) ProtoMessage() {} func (x *EmergencyReparentShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[48] + mi := &file_vtctldata_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3803,7 +3983,7 @@ func (x *EmergencyReparentShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use EmergencyReparentShardResponse.ProtoReflect.Descriptor instead. func (*EmergencyReparentShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{48} + return file_vtctldata_proto_rawDescGZIP(), []int{51} } func (x *EmergencyReparentShardResponse) GetKeyspace() string { @@ -3855,7 +4035,7 @@ type ExecuteFetchAsAppRequest struct { func (x *ExecuteFetchAsAppRequest) Reset() { *x = ExecuteFetchAsAppRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[49] + mi := &file_vtctldata_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3868,7 +4048,7 @@ func (x *ExecuteFetchAsAppRequest) String() string { func (*ExecuteFetchAsAppRequest) ProtoMessage() {} func (x *ExecuteFetchAsAppRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[49] + mi := &file_vtctldata_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3881,7 +4061,7 @@ func (x *ExecuteFetchAsAppRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteFetchAsAppRequest.ProtoReflect.Descriptor instead. func (*ExecuteFetchAsAppRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{49} + return file_vtctldata_proto_rawDescGZIP(), []int{52} } func (x *ExecuteFetchAsAppRequest) GetTabletAlias() *topodata.TabletAlias { @@ -3923,7 +4103,7 @@ type ExecuteFetchAsAppResponse struct { func (x *ExecuteFetchAsAppResponse) Reset() { *x = ExecuteFetchAsAppResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[50] + mi := &file_vtctldata_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3936,7 +4116,7 @@ func (x *ExecuteFetchAsAppResponse) String() string { func (*ExecuteFetchAsAppResponse) ProtoMessage() {} func (x *ExecuteFetchAsAppResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[50] + mi := &file_vtctldata_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3949,7 +4129,7 @@ func (x *ExecuteFetchAsAppResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteFetchAsAppResponse.ProtoReflect.Descriptor instead. func (*ExecuteFetchAsAppResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{50} + return file_vtctldata_proto_rawDescGZIP(), []int{53} } func (x *ExecuteFetchAsAppResponse) GetResult() *query.QueryResult { @@ -3984,7 +4164,7 @@ type ExecuteFetchAsDBARequest struct { func (x *ExecuteFetchAsDBARequest) Reset() { *x = ExecuteFetchAsDBARequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[51] + mi := &file_vtctldata_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3997,7 +4177,7 @@ func (x *ExecuteFetchAsDBARequest) String() string { func (*ExecuteFetchAsDBARequest) ProtoMessage() {} func (x *ExecuteFetchAsDBARequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[51] + mi := &file_vtctldata_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4010,7 +4190,7 @@ func (x *ExecuteFetchAsDBARequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteFetchAsDBARequest.ProtoReflect.Descriptor instead. func (*ExecuteFetchAsDBARequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{51} + return file_vtctldata_proto_rawDescGZIP(), []int{54} } func (x *ExecuteFetchAsDBARequest) GetTabletAlias() *topodata.TabletAlias { @@ -4059,7 +4239,7 @@ type ExecuteFetchAsDBAResponse struct { func (x *ExecuteFetchAsDBAResponse) Reset() { *x = ExecuteFetchAsDBAResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[52] + mi := &file_vtctldata_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4072,7 +4252,7 @@ func (x *ExecuteFetchAsDBAResponse) String() string { func (*ExecuteFetchAsDBAResponse) ProtoMessage() {} func (x *ExecuteFetchAsDBAResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[52] + mi := &file_vtctldata_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4085,7 +4265,7 @@ func (x *ExecuteFetchAsDBAResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteFetchAsDBAResponse.ProtoReflect.Descriptor instead. func (*ExecuteFetchAsDBAResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{52} + return file_vtctldata_proto_rawDescGZIP(), []int{55} } func (x *ExecuteFetchAsDBAResponse) GetResult() *query.QueryResult { @@ -4107,7 +4287,7 @@ type ExecuteHookRequest struct { func (x *ExecuteHookRequest) Reset() { *x = ExecuteHookRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[53] + mi := &file_vtctldata_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4120,7 +4300,7 @@ func (x *ExecuteHookRequest) String() string { func (*ExecuteHookRequest) ProtoMessage() {} func (x *ExecuteHookRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[53] + mi := &file_vtctldata_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4133,7 +4313,7 @@ func (x *ExecuteHookRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteHookRequest.ProtoReflect.Descriptor instead. func (*ExecuteHookRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{53} + return file_vtctldata_proto_rawDescGZIP(), []int{56} } func (x *ExecuteHookRequest) GetTabletAlias() *topodata.TabletAlias { @@ -4161,7 +4341,7 @@ type ExecuteHookResponse struct { func (x *ExecuteHookResponse) Reset() { *x = ExecuteHookResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[54] + mi := &file_vtctldata_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4174,7 +4354,7 @@ func (x *ExecuteHookResponse) String() string { func (*ExecuteHookResponse) ProtoMessage() {} func (x *ExecuteHookResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[54] + mi := &file_vtctldata_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4187,7 +4367,7 @@ func (x *ExecuteHookResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteHookResponse.ProtoReflect.Descriptor instead. func (*ExecuteHookResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{54} + return file_vtctldata_proto_rawDescGZIP(), []int{57} } func (x *ExecuteHookResponse) GetHookResult() *tabletmanagerdata.ExecuteHookResponse { @@ -4223,7 +4403,7 @@ type ExecuteMultiFetchAsDBARequest struct { func (x *ExecuteMultiFetchAsDBARequest) Reset() { *x = ExecuteMultiFetchAsDBARequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[55] + mi := &file_vtctldata_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4236,7 +4416,7 @@ func (x *ExecuteMultiFetchAsDBARequest) String() string { func (*ExecuteMultiFetchAsDBARequest) ProtoMessage() {} func (x *ExecuteMultiFetchAsDBARequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[55] + mi := &file_vtctldata_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4249,7 +4429,7 @@ func (x *ExecuteMultiFetchAsDBARequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteMultiFetchAsDBARequest.ProtoReflect.Descriptor instead. func (*ExecuteMultiFetchAsDBARequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{55} + return file_vtctldata_proto_rawDescGZIP(), []int{58} } func (x *ExecuteMultiFetchAsDBARequest) GetTabletAlias() *topodata.TabletAlias { @@ -4298,7 +4478,7 @@ type ExecuteMultiFetchAsDBAResponse struct { func (x *ExecuteMultiFetchAsDBAResponse) Reset() { *x = ExecuteMultiFetchAsDBAResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[56] + mi := &file_vtctldata_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4311,7 +4491,7 @@ func (x *ExecuteMultiFetchAsDBAResponse) String() string { func (*ExecuteMultiFetchAsDBAResponse) ProtoMessage() {} func (x *ExecuteMultiFetchAsDBAResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[56] + mi := &file_vtctldata_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4324,7 +4504,7 @@ func (x *ExecuteMultiFetchAsDBAResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ExecuteMultiFetchAsDBAResponse.ProtoReflect.Descriptor instead. func (*ExecuteMultiFetchAsDBAResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{56} + return file_vtctldata_proto_rawDescGZIP(), []int{59} } func (x *ExecuteMultiFetchAsDBAResponse) GetResults() []*query.QueryResult { @@ -4345,7 +4525,7 @@ type FindAllShardsInKeyspaceRequest struct { func (x *FindAllShardsInKeyspaceRequest) Reset() { *x = FindAllShardsInKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[57] + mi := &file_vtctldata_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4358,7 +4538,7 @@ func (x *FindAllShardsInKeyspaceRequest) String() string { func (*FindAllShardsInKeyspaceRequest) ProtoMessage() {} func (x *FindAllShardsInKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[57] + mi := &file_vtctldata_proto_msgTypes[60] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4371,7 +4551,7 @@ func (x *FindAllShardsInKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use FindAllShardsInKeyspaceRequest.ProtoReflect.Descriptor instead. func (*FindAllShardsInKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{57} + return file_vtctldata_proto_rawDescGZIP(), []int{60} } func (x *FindAllShardsInKeyspaceRequest) GetKeyspace() string { @@ -4392,7 +4572,7 @@ type FindAllShardsInKeyspaceResponse struct { func (x *FindAllShardsInKeyspaceResponse) Reset() { *x = FindAllShardsInKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[58] + mi := &file_vtctldata_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4405,7 +4585,7 @@ func (x *FindAllShardsInKeyspaceResponse) String() string { func (*FindAllShardsInKeyspaceResponse) ProtoMessage() {} func (x *FindAllShardsInKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[58] + mi := &file_vtctldata_proto_msgTypes[61] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4418,7 +4598,7 @@ func (x *FindAllShardsInKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use FindAllShardsInKeyspaceResponse.ProtoReflect.Descriptor instead. func (*FindAllShardsInKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{58} + return file_vtctldata_proto_rawDescGZIP(), []int{61} } func (x *FindAllShardsInKeyspaceResponse) GetShards() map[string]*Shard { @@ -4440,7 +4620,7 @@ type ForceCutOverSchemaMigrationRequest struct { func (x *ForceCutOverSchemaMigrationRequest) Reset() { *x = ForceCutOverSchemaMigrationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[59] + mi := &file_vtctldata_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4453,7 +4633,7 @@ func (x *ForceCutOverSchemaMigrationRequest) String() string { func (*ForceCutOverSchemaMigrationRequest) ProtoMessage() {} func (x *ForceCutOverSchemaMigrationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[59] + mi := &file_vtctldata_proto_msgTypes[62] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4466,7 +4646,7 @@ func (x *ForceCutOverSchemaMigrationRequest) ProtoReflect() protoreflect.Message // Deprecated: Use ForceCutOverSchemaMigrationRequest.ProtoReflect.Descriptor instead. func (*ForceCutOverSchemaMigrationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{59} + return file_vtctldata_proto_rawDescGZIP(), []int{62} } func (x *ForceCutOverSchemaMigrationRequest) GetKeyspace() string { @@ -4494,7 +4674,7 @@ type ForceCutOverSchemaMigrationResponse struct { func (x *ForceCutOverSchemaMigrationResponse) Reset() { *x = ForceCutOverSchemaMigrationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[60] + mi := &file_vtctldata_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4507,7 +4687,7 @@ func (x *ForceCutOverSchemaMigrationResponse) String() string { func (*ForceCutOverSchemaMigrationResponse) ProtoMessage() {} func (x *ForceCutOverSchemaMigrationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[60] + mi := &file_vtctldata_proto_msgTypes[63] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4520,7 +4700,7 @@ func (x *ForceCutOverSchemaMigrationResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use ForceCutOverSchemaMigrationResponse.ProtoReflect.Descriptor instead. func (*ForceCutOverSchemaMigrationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{60} + return file_vtctldata_proto_rawDescGZIP(), []int{63} } func (x *ForceCutOverSchemaMigrationResponse) GetRowsAffectedByShard() map[string]uint64 { @@ -4556,7 +4736,7 @@ type GetBackupsRequest struct { func (x *GetBackupsRequest) Reset() { *x = GetBackupsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[61] + mi := &file_vtctldata_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4569,7 +4749,7 @@ func (x *GetBackupsRequest) String() string { func (*GetBackupsRequest) ProtoMessage() {} func (x *GetBackupsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[61] + mi := &file_vtctldata_proto_msgTypes[64] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4582,7 +4762,7 @@ func (x *GetBackupsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetBackupsRequest.ProtoReflect.Descriptor instead. func (*GetBackupsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{61} + return file_vtctldata_proto_rawDescGZIP(), []int{64} } func (x *GetBackupsRequest) GetKeyspace() string { @@ -4631,7 +4811,7 @@ type GetBackupsResponse struct { func (x *GetBackupsResponse) Reset() { *x = GetBackupsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[62] + mi := &file_vtctldata_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4644,7 +4824,7 @@ func (x *GetBackupsResponse) String() string { func (*GetBackupsResponse) ProtoMessage() {} func (x *GetBackupsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[62] + mi := &file_vtctldata_proto_msgTypes[65] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4657,7 +4837,7 @@ func (x *GetBackupsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetBackupsResponse.ProtoReflect.Descriptor instead. func (*GetBackupsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{62} + return file_vtctldata_proto_rawDescGZIP(), []int{65} } func (x *GetBackupsResponse) GetBackups() []*mysqlctl.BackupInfo { @@ -4678,7 +4858,7 @@ type GetCellInfoRequest struct { func (x *GetCellInfoRequest) Reset() { *x = GetCellInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[63] + mi := &file_vtctldata_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4691,7 +4871,7 @@ func (x *GetCellInfoRequest) String() string { func (*GetCellInfoRequest) ProtoMessage() {} func (x *GetCellInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[63] + mi := &file_vtctldata_proto_msgTypes[66] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4704,7 +4884,7 @@ func (x *GetCellInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCellInfoRequest.ProtoReflect.Descriptor instead. func (*GetCellInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{63} + return file_vtctldata_proto_rawDescGZIP(), []int{66} } func (x *GetCellInfoRequest) GetCell() string { @@ -4725,7 +4905,7 @@ type GetCellInfoResponse struct { func (x *GetCellInfoResponse) Reset() { *x = GetCellInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[64] + mi := &file_vtctldata_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4738,7 +4918,7 @@ func (x *GetCellInfoResponse) String() string { func (*GetCellInfoResponse) ProtoMessage() {} func (x *GetCellInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[64] + mi := &file_vtctldata_proto_msgTypes[67] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4751,7 +4931,7 @@ func (x *GetCellInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCellInfoResponse.ProtoReflect.Descriptor instead. func (*GetCellInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{64} + return file_vtctldata_proto_rawDescGZIP(), []int{67} } func (x *GetCellInfoResponse) GetCellInfo() *topodata.CellInfo { @@ -4770,7 +4950,7 @@ type GetCellInfoNamesRequest struct { func (x *GetCellInfoNamesRequest) Reset() { *x = GetCellInfoNamesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[65] + mi := &file_vtctldata_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4783,7 +4963,7 @@ func (x *GetCellInfoNamesRequest) String() string { func (*GetCellInfoNamesRequest) ProtoMessage() {} func (x *GetCellInfoNamesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[65] + mi := &file_vtctldata_proto_msgTypes[68] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4796,7 +4976,7 @@ func (x *GetCellInfoNamesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCellInfoNamesRequest.ProtoReflect.Descriptor instead. func (*GetCellInfoNamesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{65} + return file_vtctldata_proto_rawDescGZIP(), []int{68} } type GetCellInfoNamesResponse struct { @@ -4810,7 +4990,7 @@ type GetCellInfoNamesResponse struct { func (x *GetCellInfoNamesResponse) Reset() { *x = GetCellInfoNamesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[66] + mi := &file_vtctldata_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4823,7 +5003,7 @@ func (x *GetCellInfoNamesResponse) String() string { func (*GetCellInfoNamesResponse) ProtoMessage() {} func (x *GetCellInfoNamesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[66] + mi := &file_vtctldata_proto_msgTypes[69] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4836,7 +5016,7 @@ func (x *GetCellInfoNamesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCellInfoNamesResponse.ProtoReflect.Descriptor instead. func (*GetCellInfoNamesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{66} + return file_vtctldata_proto_rawDescGZIP(), []int{69} } func (x *GetCellInfoNamesResponse) GetNames() []string { @@ -4855,7 +5035,7 @@ type GetCellsAliasesRequest struct { func (x *GetCellsAliasesRequest) Reset() { *x = GetCellsAliasesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[67] + mi := &file_vtctldata_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4868,7 +5048,7 @@ func (x *GetCellsAliasesRequest) String() string { func (*GetCellsAliasesRequest) ProtoMessage() {} func (x *GetCellsAliasesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[67] + mi := &file_vtctldata_proto_msgTypes[70] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4881,7 +5061,7 @@ func (x *GetCellsAliasesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCellsAliasesRequest.ProtoReflect.Descriptor instead. func (*GetCellsAliasesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{67} + return file_vtctldata_proto_rawDescGZIP(), []int{70} } type GetCellsAliasesResponse struct { @@ -4895,7 +5075,7 @@ type GetCellsAliasesResponse struct { func (x *GetCellsAliasesResponse) Reset() { *x = GetCellsAliasesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[68] + mi := &file_vtctldata_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4908,7 +5088,7 @@ func (x *GetCellsAliasesResponse) String() string { func (*GetCellsAliasesResponse) ProtoMessage() {} func (x *GetCellsAliasesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[68] + mi := &file_vtctldata_proto_msgTypes[71] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4921,7 +5101,7 @@ func (x *GetCellsAliasesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCellsAliasesResponse.ProtoReflect.Descriptor instead. func (*GetCellsAliasesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{68} + return file_vtctldata_proto_rawDescGZIP(), []int{71} } func (x *GetCellsAliasesResponse) GetAliases() map[string]*topodata.CellsAlias { @@ -4942,7 +5122,7 @@ type GetFullStatusRequest struct { func (x *GetFullStatusRequest) Reset() { *x = GetFullStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[69] + mi := &file_vtctldata_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4955,7 +5135,7 @@ func (x *GetFullStatusRequest) String() string { func (*GetFullStatusRequest) ProtoMessage() {} func (x *GetFullStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[69] + mi := &file_vtctldata_proto_msgTypes[72] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4968,7 +5148,7 @@ func (x *GetFullStatusRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetFullStatusRequest.ProtoReflect.Descriptor instead. func (*GetFullStatusRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{69} + return file_vtctldata_proto_rawDescGZIP(), []int{72} } func (x *GetFullStatusRequest) GetTabletAlias() *topodata.TabletAlias { @@ -4989,7 +5169,7 @@ type GetFullStatusResponse struct { func (x *GetFullStatusResponse) Reset() { *x = GetFullStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[70] + mi := &file_vtctldata_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5002,7 +5182,7 @@ func (x *GetFullStatusResponse) String() string { func (*GetFullStatusResponse) ProtoMessage() {} func (x *GetFullStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[70] + mi := &file_vtctldata_proto_msgTypes[73] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5015,7 +5195,7 @@ func (x *GetFullStatusResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetFullStatusResponse.ProtoReflect.Descriptor instead. func (*GetFullStatusResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{70} + return file_vtctldata_proto_rawDescGZIP(), []int{73} } func (x *GetFullStatusResponse) GetStatus() *replicationdata.FullStatus { @@ -5034,7 +5214,7 @@ type GetKeyspacesRequest struct { func (x *GetKeyspacesRequest) Reset() { *x = GetKeyspacesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[71] + mi := &file_vtctldata_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5047,7 +5227,7 @@ func (x *GetKeyspacesRequest) String() string { func (*GetKeyspacesRequest) ProtoMessage() {} func (x *GetKeyspacesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[71] + mi := &file_vtctldata_proto_msgTypes[74] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5060,7 +5240,7 @@ func (x *GetKeyspacesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetKeyspacesRequest.ProtoReflect.Descriptor instead. func (*GetKeyspacesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{71} + return file_vtctldata_proto_rawDescGZIP(), []int{74} } type GetKeyspacesResponse struct { @@ -5074,7 +5254,7 @@ type GetKeyspacesResponse struct { func (x *GetKeyspacesResponse) Reset() { *x = GetKeyspacesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[72] + mi := &file_vtctldata_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5087,7 +5267,7 @@ func (x *GetKeyspacesResponse) String() string { func (*GetKeyspacesResponse) ProtoMessage() {} func (x *GetKeyspacesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[72] + mi := &file_vtctldata_proto_msgTypes[75] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5100,7 +5280,7 @@ func (x *GetKeyspacesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetKeyspacesResponse.ProtoReflect.Descriptor instead. func (*GetKeyspacesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{72} + return file_vtctldata_proto_rawDescGZIP(), []int{75} } func (x *GetKeyspacesResponse) GetKeyspaces() []*Keyspace { @@ -5121,7 +5301,7 @@ type GetKeyspaceRequest struct { func (x *GetKeyspaceRequest) Reset() { *x = GetKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[73] + mi := &file_vtctldata_proto_msgTypes[76] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5134,7 +5314,7 @@ func (x *GetKeyspaceRequest) String() string { func (*GetKeyspaceRequest) ProtoMessage() {} func (x *GetKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[73] + mi := &file_vtctldata_proto_msgTypes[76] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5147,7 +5327,7 @@ func (x *GetKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetKeyspaceRequest.ProtoReflect.Descriptor instead. func (*GetKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{73} + return file_vtctldata_proto_rawDescGZIP(), []int{76} } func (x *GetKeyspaceRequest) GetKeyspace() string { @@ -5168,7 +5348,7 @@ type GetKeyspaceResponse struct { func (x *GetKeyspaceResponse) Reset() { *x = GetKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[74] + mi := &file_vtctldata_proto_msgTypes[77] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5181,7 +5361,7 @@ func (x *GetKeyspaceResponse) String() string { func (*GetKeyspaceResponse) ProtoMessage() {} func (x *GetKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[74] + mi := &file_vtctldata_proto_msgTypes[77] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5194,7 +5374,7 @@ func (x *GetKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetKeyspaceResponse.ProtoReflect.Descriptor instead. func (*GetKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{74} + return file_vtctldata_proto_rawDescGZIP(), []int{77} } func (x *GetKeyspaceResponse) GetKeyspace() *Keyspace { @@ -5215,7 +5395,7 @@ type GetPermissionsRequest struct { func (x *GetPermissionsRequest) Reset() { *x = GetPermissionsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[75] + mi := &file_vtctldata_proto_msgTypes[78] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5228,7 +5408,7 @@ func (x *GetPermissionsRequest) String() string { func (*GetPermissionsRequest) ProtoMessage() {} func (x *GetPermissionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[75] + mi := &file_vtctldata_proto_msgTypes[78] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5241,7 +5421,7 @@ func (x *GetPermissionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetPermissionsRequest.ProtoReflect.Descriptor instead. func (*GetPermissionsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{75} + return file_vtctldata_proto_rawDescGZIP(), []int{78} } func (x *GetPermissionsRequest) GetTabletAlias() *topodata.TabletAlias { @@ -5262,7 +5442,7 @@ type GetPermissionsResponse struct { func (x *GetPermissionsResponse) Reset() { *x = GetPermissionsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[76] + mi := &file_vtctldata_proto_msgTypes[79] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5275,7 +5455,7 @@ func (x *GetPermissionsResponse) String() string { func (*GetPermissionsResponse) ProtoMessage() {} func (x *GetPermissionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[76] + mi := &file_vtctldata_proto_msgTypes[79] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5288,7 +5468,7 @@ func (x *GetPermissionsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetPermissionsResponse.ProtoReflect.Descriptor instead. func (*GetPermissionsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{76} + return file_vtctldata_proto_rawDescGZIP(), []int{79} } func (x *GetPermissionsResponse) GetPermissions() *tabletmanagerdata.Permissions { @@ -5298,6 +5478,91 @@ func (x *GetPermissionsResponse) GetPermissions() *tabletmanagerdata.Permissions return nil } +type GetKeyspaceRoutingRulesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetKeyspaceRoutingRulesRequest) Reset() { + *x = GetKeyspaceRoutingRulesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetKeyspaceRoutingRulesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetKeyspaceRoutingRulesRequest) ProtoMessage() {} + +func (x *GetKeyspaceRoutingRulesRequest) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[80] + 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 GetKeyspaceRoutingRulesRequest.ProtoReflect.Descriptor instead. +func (*GetKeyspaceRoutingRulesRequest) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{80} +} + +type GetKeyspaceRoutingRulesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + KeyspaceRoutingRules *vschema.KeyspaceRoutingRules `protobuf:"bytes,1,opt,name=keyspace_routing_rules,json=keyspaceRoutingRules,proto3" json:"keyspace_routing_rules,omitempty"` +} + +func (x *GetKeyspaceRoutingRulesResponse) Reset() { + *x = GetKeyspaceRoutingRulesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_vtctldata_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetKeyspaceRoutingRulesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetKeyspaceRoutingRulesResponse) ProtoMessage() {} + +func (x *GetKeyspaceRoutingRulesResponse) ProtoReflect() protoreflect.Message { + mi := &file_vtctldata_proto_msgTypes[81] + 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 GetKeyspaceRoutingRulesResponse.ProtoReflect.Descriptor instead. +func (*GetKeyspaceRoutingRulesResponse) Descriptor() ([]byte, []int) { + return file_vtctldata_proto_rawDescGZIP(), []int{81} +} + +func (x *GetKeyspaceRoutingRulesResponse) GetKeyspaceRoutingRules() *vschema.KeyspaceRoutingRules { + if x != nil { + return x.KeyspaceRoutingRules + } + return nil +} + type GetRoutingRulesRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5307,7 +5572,7 @@ type GetRoutingRulesRequest struct { func (x *GetRoutingRulesRequest) Reset() { *x = GetRoutingRulesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[77] + mi := &file_vtctldata_proto_msgTypes[82] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5320,7 +5585,7 @@ func (x *GetRoutingRulesRequest) String() string { func (*GetRoutingRulesRequest) ProtoMessage() {} func (x *GetRoutingRulesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[77] + mi := &file_vtctldata_proto_msgTypes[82] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5333,7 +5598,7 @@ func (x *GetRoutingRulesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetRoutingRulesRequest.ProtoReflect.Descriptor instead. func (*GetRoutingRulesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{77} + return file_vtctldata_proto_rawDescGZIP(), []int{82} } type GetRoutingRulesResponse struct { @@ -5347,7 +5612,7 @@ type GetRoutingRulesResponse struct { func (x *GetRoutingRulesResponse) Reset() { *x = GetRoutingRulesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[78] + mi := &file_vtctldata_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5360,7 +5625,7 @@ func (x *GetRoutingRulesResponse) String() string { func (*GetRoutingRulesResponse) ProtoMessage() {} func (x *GetRoutingRulesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[78] + mi := &file_vtctldata_proto_msgTypes[83] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5373,7 +5638,7 @@ func (x *GetRoutingRulesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetRoutingRulesResponse.ProtoReflect.Descriptor instead. func (*GetRoutingRulesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{78} + return file_vtctldata_proto_rawDescGZIP(), []int{83} } func (x *GetRoutingRulesResponse) GetRoutingRules() *vschema.RoutingRules { @@ -5412,7 +5677,7 @@ type GetSchemaRequest struct { func (x *GetSchemaRequest) Reset() { *x = GetSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[79] + mi := &file_vtctldata_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5425,7 +5690,7 @@ func (x *GetSchemaRequest) String() string { func (*GetSchemaRequest) ProtoMessage() {} func (x *GetSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[79] + mi := &file_vtctldata_proto_msgTypes[84] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5438,7 +5703,7 @@ func (x *GetSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSchemaRequest.ProtoReflect.Descriptor instead. func (*GetSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{79} + return file_vtctldata_proto_rawDescGZIP(), []int{84} } func (x *GetSchemaRequest) GetTabletAlias() *topodata.TabletAlias { @@ -5501,7 +5766,7 @@ type GetSchemaResponse struct { func (x *GetSchemaResponse) Reset() { *x = GetSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[80] + mi := &file_vtctldata_proto_msgTypes[85] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5514,7 +5779,7 @@ func (x *GetSchemaResponse) String() string { func (*GetSchemaResponse) ProtoMessage() {} func (x *GetSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[80] + mi := &file_vtctldata_proto_msgTypes[85] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5527,7 +5792,7 @@ func (x *GetSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSchemaResponse.ProtoReflect.Descriptor instead. func (*GetSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{80} + return file_vtctldata_proto_rawDescGZIP(), []int{85} } func (x *GetSchemaResponse) GetSchema() *tabletmanagerdata.SchemaDefinition { @@ -5573,7 +5838,7 @@ type GetSchemaMigrationsRequest struct { func (x *GetSchemaMigrationsRequest) Reset() { *x = GetSchemaMigrationsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[81] + mi := &file_vtctldata_proto_msgTypes[86] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5586,7 +5851,7 @@ func (x *GetSchemaMigrationsRequest) String() string { func (*GetSchemaMigrationsRequest) ProtoMessage() {} func (x *GetSchemaMigrationsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[81] + mi := &file_vtctldata_proto_msgTypes[86] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5599,7 +5864,7 @@ func (x *GetSchemaMigrationsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSchemaMigrationsRequest.ProtoReflect.Descriptor instead. func (*GetSchemaMigrationsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{81} + return file_vtctldata_proto_rawDescGZIP(), []int{86} } func (x *GetSchemaMigrationsRequest) GetKeyspace() string { @@ -5669,7 +5934,7 @@ type GetSchemaMigrationsResponse struct { func (x *GetSchemaMigrationsResponse) Reset() { *x = GetSchemaMigrationsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[82] + mi := &file_vtctldata_proto_msgTypes[87] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5682,7 +5947,7 @@ func (x *GetSchemaMigrationsResponse) String() string { func (*GetSchemaMigrationsResponse) ProtoMessage() {} func (x *GetSchemaMigrationsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[82] + mi := &file_vtctldata_proto_msgTypes[87] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5695,7 +5960,7 @@ func (x *GetSchemaMigrationsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSchemaMigrationsResponse.ProtoReflect.Descriptor instead. func (*GetSchemaMigrationsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{82} + return file_vtctldata_proto_rawDescGZIP(), []int{87} } func (x *GetSchemaMigrationsResponse) GetMigrations() []*SchemaMigration { @@ -5720,7 +5985,7 @@ type GetShardReplicationRequest struct { func (x *GetShardReplicationRequest) Reset() { *x = GetShardReplicationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[83] + mi := &file_vtctldata_proto_msgTypes[88] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5733,7 +5998,7 @@ func (x *GetShardReplicationRequest) String() string { func (*GetShardReplicationRequest) ProtoMessage() {} func (x *GetShardReplicationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[83] + mi := &file_vtctldata_proto_msgTypes[88] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5746,7 +6011,7 @@ func (x *GetShardReplicationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetShardReplicationRequest.ProtoReflect.Descriptor instead. func (*GetShardReplicationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{83} + return file_vtctldata_proto_rawDescGZIP(), []int{88} } func (x *GetShardReplicationRequest) GetKeyspace() string { @@ -5781,7 +6046,7 @@ type GetShardReplicationResponse struct { func (x *GetShardReplicationResponse) Reset() { *x = GetShardReplicationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[84] + mi := &file_vtctldata_proto_msgTypes[89] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5794,7 +6059,7 @@ func (x *GetShardReplicationResponse) String() string { func (*GetShardReplicationResponse) ProtoMessage() {} func (x *GetShardReplicationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[84] + mi := &file_vtctldata_proto_msgTypes[89] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5807,7 +6072,7 @@ func (x *GetShardReplicationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetShardReplicationResponse.ProtoReflect.Descriptor instead. func (*GetShardReplicationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{84} + return file_vtctldata_proto_rawDescGZIP(), []int{89} } func (x *GetShardReplicationResponse) GetShardReplicationByCell() map[string]*topodata.ShardReplication { @@ -5829,7 +6094,7 @@ type GetShardRequest struct { func (x *GetShardRequest) Reset() { *x = GetShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[85] + mi := &file_vtctldata_proto_msgTypes[90] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5842,7 +6107,7 @@ func (x *GetShardRequest) String() string { func (*GetShardRequest) ProtoMessage() {} func (x *GetShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[85] + mi := &file_vtctldata_proto_msgTypes[90] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5855,7 +6120,7 @@ func (x *GetShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetShardRequest.ProtoReflect.Descriptor instead. func (*GetShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{85} + return file_vtctldata_proto_rawDescGZIP(), []int{90} } func (x *GetShardRequest) GetKeyspace() string { @@ -5883,7 +6148,7 @@ type GetShardResponse struct { func (x *GetShardResponse) Reset() { *x = GetShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[86] + mi := &file_vtctldata_proto_msgTypes[91] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5896,7 +6161,7 @@ func (x *GetShardResponse) String() string { func (*GetShardResponse) ProtoMessage() {} func (x *GetShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[86] + mi := &file_vtctldata_proto_msgTypes[91] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5909,7 +6174,7 @@ func (x *GetShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetShardResponse.ProtoReflect.Descriptor instead. func (*GetShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{86} + return file_vtctldata_proto_rawDescGZIP(), []int{91} } func (x *GetShardResponse) GetShard() *Shard { @@ -5928,7 +6193,7 @@ type GetShardRoutingRulesRequest struct { func (x *GetShardRoutingRulesRequest) Reset() { *x = GetShardRoutingRulesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[87] + mi := &file_vtctldata_proto_msgTypes[92] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5941,7 +6206,7 @@ func (x *GetShardRoutingRulesRequest) String() string { func (*GetShardRoutingRulesRequest) ProtoMessage() {} func (x *GetShardRoutingRulesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[87] + mi := &file_vtctldata_proto_msgTypes[92] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5954,7 +6219,7 @@ func (x *GetShardRoutingRulesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetShardRoutingRulesRequest.ProtoReflect.Descriptor instead. func (*GetShardRoutingRulesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{87} + return file_vtctldata_proto_rawDescGZIP(), []int{92} } type GetShardRoutingRulesResponse struct { @@ -5968,7 +6233,7 @@ type GetShardRoutingRulesResponse struct { func (x *GetShardRoutingRulesResponse) Reset() { *x = GetShardRoutingRulesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[88] + mi := &file_vtctldata_proto_msgTypes[93] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5981,7 +6246,7 @@ func (x *GetShardRoutingRulesResponse) String() string { func (*GetShardRoutingRulesResponse) ProtoMessage() {} func (x *GetShardRoutingRulesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[88] + mi := &file_vtctldata_proto_msgTypes[93] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5994,7 +6259,7 @@ func (x *GetShardRoutingRulesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetShardRoutingRulesResponse.ProtoReflect.Descriptor instead. func (*GetShardRoutingRulesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{88} + return file_vtctldata_proto_rawDescGZIP(), []int{93} } func (x *GetShardRoutingRulesResponse) GetShardRoutingRules() *vschema.ShardRoutingRules { @@ -6015,7 +6280,7 @@ type GetSrvKeyspaceNamesRequest struct { func (x *GetSrvKeyspaceNamesRequest) Reset() { *x = GetSrvKeyspaceNamesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[89] + mi := &file_vtctldata_proto_msgTypes[94] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6028,7 +6293,7 @@ func (x *GetSrvKeyspaceNamesRequest) String() string { func (*GetSrvKeyspaceNamesRequest) ProtoMessage() {} func (x *GetSrvKeyspaceNamesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[89] + mi := &file_vtctldata_proto_msgTypes[94] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6041,7 +6306,7 @@ func (x *GetSrvKeyspaceNamesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvKeyspaceNamesRequest.ProtoReflect.Descriptor instead. func (*GetSrvKeyspaceNamesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{89} + return file_vtctldata_proto_rawDescGZIP(), []int{94} } func (x *GetSrvKeyspaceNamesRequest) GetCells() []string { @@ -6063,7 +6328,7 @@ type GetSrvKeyspaceNamesResponse struct { func (x *GetSrvKeyspaceNamesResponse) Reset() { *x = GetSrvKeyspaceNamesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[90] + mi := &file_vtctldata_proto_msgTypes[95] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6076,7 +6341,7 @@ func (x *GetSrvKeyspaceNamesResponse) String() string { func (*GetSrvKeyspaceNamesResponse) ProtoMessage() {} func (x *GetSrvKeyspaceNamesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[90] + mi := &file_vtctldata_proto_msgTypes[95] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6089,7 +6354,7 @@ func (x *GetSrvKeyspaceNamesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvKeyspaceNamesResponse.ProtoReflect.Descriptor instead. func (*GetSrvKeyspaceNamesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{90} + return file_vtctldata_proto_rawDescGZIP(), []int{95} } func (x *GetSrvKeyspaceNamesResponse) GetNames() map[string]*GetSrvKeyspaceNamesResponse_NameList { @@ -6113,7 +6378,7 @@ type GetSrvKeyspacesRequest struct { func (x *GetSrvKeyspacesRequest) Reset() { *x = GetSrvKeyspacesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[91] + mi := &file_vtctldata_proto_msgTypes[96] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6126,7 +6391,7 @@ func (x *GetSrvKeyspacesRequest) String() string { func (*GetSrvKeyspacesRequest) ProtoMessage() {} func (x *GetSrvKeyspacesRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[91] + mi := &file_vtctldata_proto_msgTypes[96] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6139,7 +6404,7 @@ func (x *GetSrvKeyspacesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvKeyspacesRequest.ProtoReflect.Descriptor instead. func (*GetSrvKeyspacesRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{91} + return file_vtctldata_proto_rawDescGZIP(), []int{96} } func (x *GetSrvKeyspacesRequest) GetKeyspace() string { @@ -6168,7 +6433,7 @@ type GetSrvKeyspacesResponse struct { func (x *GetSrvKeyspacesResponse) Reset() { *x = GetSrvKeyspacesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[92] + mi := &file_vtctldata_proto_msgTypes[97] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6181,7 +6446,7 @@ func (x *GetSrvKeyspacesResponse) String() string { func (*GetSrvKeyspacesResponse) ProtoMessage() {} func (x *GetSrvKeyspacesResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[92] + mi := &file_vtctldata_proto_msgTypes[97] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6194,7 +6459,7 @@ func (x *GetSrvKeyspacesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvKeyspacesResponse.ProtoReflect.Descriptor instead. func (*GetSrvKeyspacesResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{92} + return file_vtctldata_proto_rawDescGZIP(), []int{97} } func (x *GetSrvKeyspacesResponse) GetSrvKeyspaces() map[string]*topodata.SrvKeyspace { @@ -6231,7 +6496,7 @@ type UpdateThrottlerConfigRequest struct { func (x *UpdateThrottlerConfigRequest) Reset() { *x = UpdateThrottlerConfigRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[93] + mi := &file_vtctldata_proto_msgTypes[98] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6244,7 +6509,7 @@ func (x *UpdateThrottlerConfigRequest) String() string { func (*UpdateThrottlerConfigRequest) ProtoMessage() {} func (x *UpdateThrottlerConfigRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[93] + mi := &file_vtctldata_proto_msgTypes[98] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6257,7 +6522,7 @@ func (x *UpdateThrottlerConfigRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateThrottlerConfigRequest.ProtoReflect.Descriptor instead. func (*UpdateThrottlerConfigRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{93} + return file_vtctldata_proto_rawDescGZIP(), []int{98} } func (x *UpdateThrottlerConfigRequest) GetKeyspace() string { @@ -6332,7 +6597,7 @@ type UpdateThrottlerConfigResponse struct { func (x *UpdateThrottlerConfigResponse) Reset() { *x = UpdateThrottlerConfigResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[94] + mi := &file_vtctldata_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6345,7 +6610,7 @@ func (x *UpdateThrottlerConfigResponse) String() string { func (*UpdateThrottlerConfigResponse) ProtoMessage() {} func (x *UpdateThrottlerConfigResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[94] + mi := &file_vtctldata_proto_msgTypes[99] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6358,7 +6623,7 @@ func (x *UpdateThrottlerConfigResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateThrottlerConfigResponse.ProtoReflect.Descriptor instead. func (*UpdateThrottlerConfigResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{94} + return file_vtctldata_proto_rawDescGZIP(), []int{99} } type GetSrvVSchemaRequest struct { @@ -6372,7 +6637,7 @@ type GetSrvVSchemaRequest struct { func (x *GetSrvVSchemaRequest) Reset() { *x = GetSrvVSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[95] + mi := &file_vtctldata_proto_msgTypes[100] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6385,7 +6650,7 @@ func (x *GetSrvVSchemaRequest) String() string { func (*GetSrvVSchemaRequest) ProtoMessage() {} func (x *GetSrvVSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[95] + mi := &file_vtctldata_proto_msgTypes[100] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6398,7 +6663,7 @@ func (x *GetSrvVSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvVSchemaRequest.ProtoReflect.Descriptor instead. func (*GetSrvVSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{95} + return file_vtctldata_proto_rawDescGZIP(), []int{100} } func (x *GetSrvVSchemaRequest) GetCell() string { @@ -6419,7 +6684,7 @@ type GetSrvVSchemaResponse struct { func (x *GetSrvVSchemaResponse) Reset() { *x = GetSrvVSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[96] + mi := &file_vtctldata_proto_msgTypes[101] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6432,7 +6697,7 @@ func (x *GetSrvVSchemaResponse) String() string { func (*GetSrvVSchemaResponse) ProtoMessage() {} func (x *GetSrvVSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[96] + mi := &file_vtctldata_proto_msgTypes[101] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6445,7 +6710,7 @@ func (x *GetSrvVSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvVSchemaResponse.ProtoReflect.Descriptor instead. func (*GetSrvVSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{96} + return file_vtctldata_proto_rawDescGZIP(), []int{101} } func (x *GetSrvVSchemaResponse) GetSrvVSchema() *vschema.SrvVSchema { @@ -6466,7 +6731,7 @@ type GetSrvVSchemasRequest struct { func (x *GetSrvVSchemasRequest) Reset() { *x = GetSrvVSchemasRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[97] + mi := &file_vtctldata_proto_msgTypes[102] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6479,7 +6744,7 @@ func (x *GetSrvVSchemasRequest) String() string { func (*GetSrvVSchemasRequest) ProtoMessage() {} func (x *GetSrvVSchemasRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[97] + mi := &file_vtctldata_proto_msgTypes[102] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6492,7 +6757,7 @@ func (x *GetSrvVSchemasRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvVSchemasRequest.ProtoReflect.Descriptor instead. func (*GetSrvVSchemasRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{97} + return file_vtctldata_proto_rawDescGZIP(), []int{102} } func (x *GetSrvVSchemasRequest) GetCells() []string { @@ -6514,7 +6779,7 @@ type GetSrvVSchemasResponse struct { func (x *GetSrvVSchemasResponse) Reset() { *x = GetSrvVSchemasResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[98] + mi := &file_vtctldata_proto_msgTypes[103] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6527,7 +6792,7 @@ func (x *GetSrvVSchemasResponse) String() string { func (*GetSrvVSchemasResponse) ProtoMessage() {} func (x *GetSrvVSchemasResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[98] + mi := &file_vtctldata_proto_msgTypes[103] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6540,7 +6805,7 @@ func (x *GetSrvVSchemasResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetSrvVSchemasResponse.ProtoReflect.Descriptor instead. func (*GetSrvVSchemasResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{98} + return file_vtctldata_proto_rawDescGZIP(), []int{103} } func (x *GetSrvVSchemasResponse) GetSrvVSchemas() map[string]*vschema.SrvVSchema { @@ -6561,7 +6826,7 @@ type GetTabletRequest struct { func (x *GetTabletRequest) Reset() { *x = GetTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[99] + mi := &file_vtctldata_proto_msgTypes[104] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6574,7 +6839,7 @@ func (x *GetTabletRequest) String() string { func (*GetTabletRequest) ProtoMessage() {} func (x *GetTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[99] + mi := &file_vtctldata_proto_msgTypes[104] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6587,7 +6852,7 @@ func (x *GetTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTabletRequest.ProtoReflect.Descriptor instead. func (*GetTabletRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{99} + return file_vtctldata_proto_rawDescGZIP(), []int{104} } func (x *GetTabletRequest) GetTabletAlias() *topodata.TabletAlias { @@ -6608,7 +6873,7 @@ type GetTabletResponse struct { func (x *GetTabletResponse) Reset() { *x = GetTabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[100] + mi := &file_vtctldata_proto_msgTypes[105] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6621,7 +6886,7 @@ func (x *GetTabletResponse) String() string { func (*GetTabletResponse) ProtoMessage() {} func (x *GetTabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[100] + mi := &file_vtctldata_proto_msgTypes[105] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6634,7 +6899,7 @@ func (x *GetTabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTabletResponse.ProtoReflect.Descriptor instead. func (*GetTabletResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{100} + return file_vtctldata_proto_rawDescGZIP(), []int{105} } func (x *GetTabletResponse) GetTablet() *topodata.Tablet { @@ -6676,7 +6941,7 @@ type GetTabletsRequest struct { func (x *GetTabletsRequest) Reset() { *x = GetTabletsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[101] + mi := &file_vtctldata_proto_msgTypes[106] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6689,7 +6954,7 @@ func (x *GetTabletsRequest) String() string { func (*GetTabletsRequest) ProtoMessage() {} func (x *GetTabletsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[101] + mi := &file_vtctldata_proto_msgTypes[106] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6702,7 +6967,7 @@ func (x *GetTabletsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTabletsRequest.ProtoReflect.Descriptor instead. func (*GetTabletsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{101} + return file_vtctldata_proto_rawDescGZIP(), []int{106} } func (x *GetTabletsRequest) GetKeyspace() string { @@ -6758,7 +7023,7 @@ type GetTabletsResponse struct { func (x *GetTabletsResponse) Reset() { *x = GetTabletsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[102] + mi := &file_vtctldata_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6771,7 +7036,7 @@ func (x *GetTabletsResponse) String() string { func (*GetTabletsResponse) ProtoMessage() {} func (x *GetTabletsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[102] + mi := &file_vtctldata_proto_msgTypes[107] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6784,7 +7049,7 @@ func (x *GetTabletsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTabletsResponse.ProtoReflect.Descriptor instead. func (*GetTabletsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{102} + return file_vtctldata_proto_rawDescGZIP(), []int{107} } func (x *GetTabletsResponse) GetTablets() []*topodata.Tablet { @@ -6805,7 +7070,7 @@ type GetTopologyPathRequest struct { func (x *GetTopologyPathRequest) Reset() { *x = GetTopologyPathRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[103] + mi := &file_vtctldata_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6818,7 +7083,7 @@ func (x *GetTopologyPathRequest) String() string { func (*GetTopologyPathRequest) ProtoMessage() {} func (x *GetTopologyPathRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[103] + mi := &file_vtctldata_proto_msgTypes[108] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6831,7 +7096,7 @@ func (x *GetTopologyPathRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTopologyPathRequest.ProtoReflect.Descriptor instead. func (*GetTopologyPathRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{103} + return file_vtctldata_proto_rawDescGZIP(), []int{108} } func (x *GetTopologyPathRequest) GetPath() string { @@ -6852,7 +7117,7 @@ type GetTopologyPathResponse struct { func (x *GetTopologyPathResponse) Reset() { *x = GetTopologyPathResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[104] + mi := &file_vtctldata_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6865,7 +7130,7 @@ func (x *GetTopologyPathResponse) String() string { func (*GetTopologyPathResponse) ProtoMessage() {} func (x *GetTopologyPathResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[104] + mi := &file_vtctldata_proto_msgTypes[109] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6878,7 +7143,7 @@ func (x *GetTopologyPathResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTopologyPathResponse.ProtoReflect.Descriptor instead. func (*GetTopologyPathResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{104} + return file_vtctldata_proto_rawDescGZIP(), []int{109} } func (x *GetTopologyPathResponse) GetCell() *TopologyCell { @@ -6904,7 +7169,7 @@ type TopologyCell struct { func (x *TopologyCell) Reset() { *x = TopologyCell{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[105] + mi := &file_vtctldata_proto_msgTypes[110] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6917,7 +7182,7 @@ func (x *TopologyCell) String() string { func (*TopologyCell) ProtoMessage() {} func (x *TopologyCell) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[105] + mi := &file_vtctldata_proto_msgTypes[110] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6930,7 +7195,7 @@ func (x *TopologyCell) ProtoReflect() protoreflect.Message { // Deprecated: Use TopologyCell.ProtoReflect.Descriptor instead. func (*TopologyCell) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{105} + return file_vtctldata_proto_rawDescGZIP(), []int{110} } func (x *TopologyCell) GetName() string { @@ -6972,7 +7237,7 @@ type GetVSchemaRequest struct { func (x *GetVSchemaRequest) Reset() { *x = GetVSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[106] + mi := &file_vtctldata_proto_msgTypes[111] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6985,7 +7250,7 @@ func (x *GetVSchemaRequest) String() string { func (*GetVSchemaRequest) ProtoMessage() {} func (x *GetVSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[106] + mi := &file_vtctldata_proto_msgTypes[111] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6998,7 +7263,7 @@ func (x *GetVSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetVSchemaRequest.ProtoReflect.Descriptor instead. func (*GetVSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{106} + return file_vtctldata_proto_rawDescGZIP(), []int{111} } func (x *GetVSchemaRequest) GetKeyspace() string { @@ -7019,7 +7284,7 @@ type GetVersionRequest struct { func (x *GetVersionRequest) Reset() { *x = GetVersionRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[107] + mi := &file_vtctldata_proto_msgTypes[112] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7032,7 +7297,7 @@ func (x *GetVersionRequest) String() string { func (*GetVersionRequest) ProtoMessage() {} func (x *GetVersionRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[107] + mi := &file_vtctldata_proto_msgTypes[112] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7045,7 +7310,7 @@ func (x *GetVersionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetVersionRequest.ProtoReflect.Descriptor instead. func (*GetVersionRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{107} + return file_vtctldata_proto_rawDescGZIP(), []int{112} } func (x *GetVersionRequest) GetTabletAlias() *topodata.TabletAlias { @@ -7066,7 +7331,7 @@ type GetVersionResponse struct { func (x *GetVersionResponse) Reset() { *x = GetVersionResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[108] + mi := &file_vtctldata_proto_msgTypes[113] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7079,7 +7344,7 @@ func (x *GetVersionResponse) String() string { func (*GetVersionResponse) ProtoMessage() {} func (x *GetVersionResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[108] + mi := &file_vtctldata_proto_msgTypes[113] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7092,7 +7357,7 @@ func (x *GetVersionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetVersionResponse.ProtoReflect.Descriptor instead. func (*GetVersionResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{108} + return file_vtctldata_proto_rawDescGZIP(), []int{113} } func (x *GetVersionResponse) GetVersion() string { @@ -7113,7 +7378,7 @@ type GetVSchemaResponse struct { func (x *GetVSchemaResponse) Reset() { *x = GetVSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[109] + mi := &file_vtctldata_proto_msgTypes[114] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7126,7 +7391,7 @@ func (x *GetVSchemaResponse) String() string { func (*GetVSchemaResponse) ProtoMessage() {} func (x *GetVSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[109] + mi := &file_vtctldata_proto_msgTypes[114] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7139,7 +7404,7 @@ func (x *GetVSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetVSchemaResponse.ProtoReflect.Descriptor instead. func (*GetVSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{109} + return file_vtctldata_proto_rawDescGZIP(), []int{114} } func (x *GetVSchemaResponse) GetVSchema() *vschema.Keyspace { @@ -7166,7 +7431,7 @@ type GetWorkflowsRequest struct { func (x *GetWorkflowsRequest) Reset() { *x = GetWorkflowsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[110] + mi := &file_vtctldata_proto_msgTypes[115] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7179,7 +7444,7 @@ func (x *GetWorkflowsRequest) String() string { func (*GetWorkflowsRequest) ProtoMessage() {} func (x *GetWorkflowsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[110] + mi := &file_vtctldata_proto_msgTypes[115] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7192,7 +7457,7 @@ func (x *GetWorkflowsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetWorkflowsRequest.ProtoReflect.Descriptor instead. func (*GetWorkflowsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{110} + return file_vtctldata_proto_rawDescGZIP(), []int{115} } func (x *GetWorkflowsRequest) GetKeyspace() string { @@ -7248,7 +7513,7 @@ type GetWorkflowsResponse struct { func (x *GetWorkflowsResponse) Reset() { *x = GetWorkflowsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[111] + mi := &file_vtctldata_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7261,7 +7526,7 @@ func (x *GetWorkflowsResponse) String() string { func (*GetWorkflowsResponse) ProtoMessage() {} func (x *GetWorkflowsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[111] + mi := &file_vtctldata_proto_msgTypes[116] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7274,7 +7539,7 @@ func (x *GetWorkflowsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetWorkflowsResponse.ProtoReflect.Descriptor instead. func (*GetWorkflowsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{111} + return file_vtctldata_proto_rawDescGZIP(), []int{116} } func (x *GetWorkflowsResponse) GetWorkflows() []*Workflow { @@ -7299,7 +7564,7 @@ type InitShardPrimaryRequest struct { func (x *InitShardPrimaryRequest) Reset() { *x = InitShardPrimaryRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[112] + mi := &file_vtctldata_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7312,7 +7577,7 @@ func (x *InitShardPrimaryRequest) String() string { func (*InitShardPrimaryRequest) ProtoMessage() {} func (x *InitShardPrimaryRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[112] + mi := &file_vtctldata_proto_msgTypes[117] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7325,7 +7590,7 @@ func (x *InitShardPrimaryRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use InitShardPrimaryRequest.ProtoReflect.Descriptor instead. func (*InitShardPrimaryRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{112} + return file_vtctldata_proto_rawDescGZIP(), []int{117} } func (x *InitShardPrimaryRequest) GetKeyspace() string { @@ -7374,7 +7639,7 @@ type InitShardPrimaryResponse struct { func (x *InitShardPrimaryResponse) Reset() { *x = InitShardPrimaryResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[113] + mi := &file_vtctldata_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7387,7 +7652,7 @@ func (x *InitShardPrimaryResponse) String() string { func (*InitShardPrimaryResponse) ProtoMessage() {} func (x *InitShardPrimaryResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[113] + mi := &file_vtctldata_proto_msgTypes[118] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7400,7 +7665,7 @@ func (x *InitShardPrimaryResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use InitShardPrimaryResponse.ProtoReflect.Descriptor instead. func (*InitShardPrimaryResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{113} + return file_vtctldata_proto_rawDescGZIP(), []int{118} } func (x *InitShardPrimaryResponse) GetEvents() []*logutil.Event { @@ -7422,7 +7687,7 @@ type LaunchSchemaMigrationRequest struct { func (x *LaunchSchemaMigrationRequest) Reset() { *x = LaunchSchemaMigrationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[114] + mi := &file_vtctldata_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7435,7 +7700,7 @@ func (x *LaunchSchemaMigrationRequest) String() string { func (*LaunchSchemaMigrationRequest) ProtoMessage() {} func (x *LaunchSchemaMigrationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[114] + mi := &file_vtctldata_proto_msgTypes[119] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7448,7 +7713,7 @@ func (x *LaunchSchemaMigrationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LaunchSchemaMigrationRequest.ProtoReflect.Descriptor instead. func (*LaunchSchemaMigrationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{114} + return file_vtctldata_proto_rawDescGZIP(), []int{119} } func (x *LaunchSchemaMigrationRequest) GetKeyspace() string { @@ -7476,7 +7741,7 @@ type LaunchSchemaMigrationResponse struct { func (x *LaunchSchemaMigrationResponse) Reset() { *x = LaunchSchemaMigrationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[115] + mi := &file_vtctldata_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7489,7 +7754,7 @@ func (x *LaunchSchemaMigrationResponse) String() string { func (*LaunchSchemaMigrationResponse) ProtoMessage() {} func (x *LaunchSchemaMigrationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[115] + mi := &file_vtctldata_proto_msgTypes[120] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7502,7 +7767,7 @@ func (x *LaunchSchemaMigrationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LaunchSchemaMigrationResponse.ProtoReflect.Descriptor instead. func (*LaunchSchemaMigrationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{115} + return file_vtctldata_proto_rawDescGZIP(), []int{120} } func (x *LaunchSchemaMigrationResponse) GetRowsAffectedByShard() map[string]uint64 { @@ -7529,7 +7794,7 @@ type LookupVindexCreateRequest struct { func (x *LookupVindexCreateRequest) Reset() { *x = LookupVindexCreateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[116] + mi := &file_vtctldata_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7542,7 +7807,7 @@ func (x *LookupVindexCreateRequest) String() string { func (*LookupVindexCreateRequest) ProtoMessage() {} func (x *LookupVindexCreateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[116] + mi := &file_vtctldata_proto_msgTypes[121] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7555,7 +7820,7 @@ func (x *LookupVindexCreateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVindexCreateRequest.ProtoReflect.Descriptor instead. func (*LookupVindexCreateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{116} + return file_vtctldata_proto_rawDescGZIP(), []int{121} } func (x *LookupVindexCreateRequest) GetKeyspace() string { @@ -7616,7 +7881,7 @@ type LookupVindexCreateResponse struct { func (x *LookupVindexCreateResponse) Reset() { *x = LookupVindexCreateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[117] + mi := &file_vtctldata_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7629,7 +7894,7 @@ func (x *LookupVindexCreateResponse) String() string { func (*LookupVindexCreateResponse) ProtoMessage() {} func (x *LookupVindexCreateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[117] + mi := &file_vtctldata_proto_msgTypes[122] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7642,7 +7907,7 @@ func (x *LookupVindexCreateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVindexCreateResponse.ProtoReflect.Descriptor instead. func (*LookupVindexCreateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{117} + return file_vtctldata_proto_rawDescGZIP(), []int{122} } type LookupVindexExternalizeRequest struct { @@ -7661,7 +7926,7 @@ type LookupVindexExternalizeRequest struct { func (x *LookupVindexExternalizeRequest) Reset() { *x = LookupVindexExternalizeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[118] + mi := &file_vtctldata_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7674,7 +7939,7 @@ func (x *LookupVindexExternalizeRequest) String() string { func (*LookupVindexExternalizeRequest) ProtoMessage() {} func (x *LookupVindexExternalizeRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[118] + mi := &file_vtctldata_proto_msgTypes[123] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7687,7 +7952,7 @@ func (x *LookupVindexExternalizeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVindexExternalizeRequest.ProtoReflect.Descriptor instead. func (*LookupVindexExternalizeRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{118} + return file_vtctldata_proto_rawDescGZIP(), []int{123} } func (x *LookupVindexExternalizeRequest) GetKeyspace() string { @@ -7723,7 +7988,7 @@ type LookupVindexExternalizeResponse struct { func (x *LookupVindexExternalizeResponse) Reset() { *x = LookupVindexExternalizeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[119] + mi := &file_vtctldata_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7736,7 +8001,7 @@ func (x *LookupVindexExternalizeResponse) String() string { func (*LookupVindexExternalizeResponse) ProtoMessage() {} func (x *LookupVindexExternalizeResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[119] + mi := &file_vtctldata_proto_msgTypes[124] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7749,7 +8014,7 @@ func (x *LookupVindexExternalizeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupVindexExternalizeResponse.ProtoReflect.Descriptor instead. func (*LookupVindexExternalizeResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{119} + return file_vtctldata_proto_rawDescGZIP(), []int{124} } func (x *LookupVindexExternalizeResponse) GetWorkflowDeleted() bool { @@ -7770,7 +8035,7 @@ type MaterializeCreateRequest struct { func (x *MaterializeCreateRequest) Reset() { *x = MaterializeCreateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[120] + mi := &file_vtctldata_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7783,7 +8048,7 @@ func (x *MaterializeCreateRequest) String() string { func (*MaterializeCreateRequest) ProtoMessage() {} func (x *MaterializeCreateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[120] + mi := &file_vtctldata_proto_msgTypes[125] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7796,7 +8061,7 @@ func (x *MaterializeCreateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MaterializeCreateRequest.ProtoReflect.Descriptor instead. func (*MaterializeCreateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{120} + return file_vtctldata_proto_rawDescGZIP(), []int{125} } func (x *MaterializeCreateRequest) GetSettings() *MaterializeSettings { @@ -7815,7 +8080,7 @@ type MaterializeCreateResponse struct { func (x *MaterializeCreateResponse) Reset() { *x = MaterializeCreateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[121] + mi := &file_vtctldata_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7828,7 +8093,7 @@ func (x *MaterializeCreateResponse) String() string { func (*MaterializeCreateResponse) ProtoMessage() {} func (x *MaterializeCreateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[121] + mi := &file_vtctldata_proto_msgTypes[126] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7841,7 +8106,7 @@ func (x *MaterializeCreateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MaterializeCreateResponse.ProtoReflect.Descriptor instead. func (*MaterializeCreateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{121} + return file_vtctldata_proto_rawDescGZIP(), []int{126} } type MigrateCreateRequest struct { @@ -7880,7 +8145,7 @@ type MigrateCreateRequest struct { func (x *MigrateCreateRequest) Reset() { *x = MigrateCreateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[122] + mi := &file_vtctldata_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7893,7 +8158,7 @@ func (x *MigrateCreateRequest) String() string { func (*MigrateCreateRequest) ProtoMessage() {} func (x *MigrateCreateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[122] + mi := &file_vtctldata_proto_msgTypes[127] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7906,7 +8171,7 @@ func (x *MigrateCreateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MigrateCreateRequest.ProtoReflect.Descriptor instead. func (*MigrateCreateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{122} + return file_vtctldata_proto_rawDescGZIP(), []int{127} } func (x *MigrateCreateRequest) GetWorkflow() string { @@ -8044,7 +8309,7 @@ type MigrateCompleteRequest struct { func (x *MigrateCompleteRequest) Reset() { *x = MigrateCompleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[123] + mi := &file_vtctldata_proto_msgTypes[128] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8057,7 +8322,7 @@ func (x *MigrateCompleteRequest) String() string { func (*MigrateCompleteRequest) ProtoMessage() {} func (x *MigrateCompleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[123] + mi := &file_vtctldata_proto_msgTypes[128] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8070,7 +8335,7 @@ func (x *MigrateCompleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MigrateCompleteRequest.ProtoReflect.Descriptor instead. func (*MigrateCompleteRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{123} + return file_vtctldata_proto_rawDescGZIP(), []int{128} } func (x *MigrateCompleteRequest) GetWorkflow() string { @@ -8127,7 +8392,7 @@ type MigrateCompleteResponse struct { func (x *MigrateCompleteResponse) Reset() { *x = MigrateCompleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[124] + mi := &file_vtctldata_proto_msgTypes[129] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8140,7 +8405,7 @@ func (x *MigrateCompleteResponse) String() string { func (*MigrateCompleteResponse) ProtoMessage() {} func (x *MigrateCompleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[124] + mi := &file_vtctldata_proto_msgTypes[129] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8153,7 +8418,7 @@ func (x *MigrateCompleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MigrateCompleteResponse.ProtoReflect.Descriptor instead. func (*MigrateCompleteResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{124} + return file_vtctldata_proto_rawDescGZIP(), []int{129} } func (x *MigrateCompleteResponse) GetSummary() string { @@ -8184,7 +8449,7 @@ type MountRegisterRequest struct { func (x *MountRegisterRequest) Reset() { *x = MountRegisterRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[125] + mi := &file_vtctldata_proto_msgTypes[130] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8197,7 +8462,7 @@ func (x *MountRegisterRequest) String() string { func (*MountRegisterRequest) ProtoMessage() {} func (x *MountRegisterRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[125] + mi := &file_vtctldata_proto_msgTypes[130] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8210,7 +8475,7 @@ func (x *MountRegisterRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MountRegisterRequest.ProtoReflect.Descriptor instead. func (*MountRegisterRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{125} + return file_vtctldata_proto_rawDescGZIP(), []int{130} } func (x *MountRegisterRequest) GetTopoType() string { @@ -8250,7 +8515,7 @@ type MountRegisterResponse struct { func (x *MountRegisterResponse) Reset() { *x = MountRegisterResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[126] + mi := &file_vtctldata_proto_msgTypes[131] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8263,7 +8528,7 @@ func (x *MountRegisterResponse) String() string { func (*MountRegisterResponse) ProtoMessage() {} func (x *MountRegisterResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[126] + mi := &file_vtctldata_proto_msgTypes[131] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8276,7 +8541,7 @@ func (x *MountRegisterResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MountRegisterResponse.ProtoReflect.Descriptor instead. func (*MountRegisterResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{126} + return file_vtctldata_proto_rawDescGZIP(), []int{131} } type MountUnregisterRequest struct { @@ -8290,7 +8555,7 @@ type MountUnregisterRequest struct { func (x *MountUnregisterRequest) Reset() { *x = MountUnregisterRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[127] + mi := &file_vtctldata_proto_msgTypes[132] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8303,7 +8568,7 @@ func (x *MountUnregisterRequest) String() string { func (*MountUnregisterRequest) ProtoMessage() {} func (x *MountUnregisterRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[127] + mi := &file_vtctldata_proto_msgTypes[132] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8316,7 +8581,7 @@ func (x *MountUnregisterRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MountUnregisterRequest.ProtoReflect.Descriptor instead. func (*MountUnregisterRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{127} + return file_vtctldata_proto_rawDescGZIP(), []int{132} } func (x *MountUnregisterRequest) GetName() string { @@ -8335,7 +8600,7 @@ type MountUnregisterResponse struct { func (x *MountUnregisterResponse) Reset() { *x = MountUnregisterResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[128] + mi := &file_vtctldata_proto_msgTypes[133] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8348,7 +8613,7 @@ func (x *MountUnregisterResponse) String() string { func (*MountUnregisterResponse) ProtoMessage() {} func (x *MountUnregisterResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[128] + mi := &file_vtctldata_proto_msgTypes[133] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8361,7 +8626,7 @@ func (x *MountUnregisterResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MountUnregisterResponse.ProtoReflect.Descriptor instead. func (*MountUnregisterResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{128} + return file_vtctldata_proto_rawDescGZIP(), []int{133} } type MountShowRequest struct { @@ -8375,7 +8640,7 @@ type MountShowRequest struct { func (x *MountShowRequest) Reset() { *x = MountShowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[129] + mi := &file_vtctldata_proto_msgTypes[134] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8388,7 +8653,7 @@ func (x *MountShowRequest) String() string { func (*MountShowRequest) ProtoMessage() {} func (x *MountShowRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[129] + mi := &file_vtctldata_proto_msgTypes[134] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8401,7 +8666,7 @@ func (x *MountShowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MountShowRequest.ProtoReflect.Descriptor instead. func (*MountShowRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{129} + return file_vtctldata_proto_rawDescGZIP(), []int{134} } func (x *MountShowRequest) GetName() string { @@ -8425,7 +8690,7 @@ type MountShowResponse struct { func (x *MountShowResponse) Reset() { *x = MountShowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[130] + mi := &file_vtctldata_proto_msgTypes[135] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8438,7 +8703,7 @@ func (x *MountShowResponse) String() string { func (*MountShowResponse) ProtoMessage() {} func (x *MountShowResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[130] + mi := &file_vtctldata_proto_msgTypes[135] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8451,7 +8716,7 @@ func (x *MountShowResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MountShowResponse.ProtoReflect.Descriptor instead. func (*MountShowResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{130} + return file_vtctldata_proto_rawDescGZIP(), []int{135} } func (x *MountShowResponse) GetTopoType() string { @@ -8491,7 +8756,7 @@ type MountListRequest struct { func (x *MountListRequest) Reset() { *x = MountListRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[131] + mi := &file_vtctldata_proto_msgTypes[136] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8504,7 +8769,7 @@ func (x *MountListRequest) String() string { func (*MountListRequest) ProtoMessage() {} func (x *MountListRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[131] + mi := &file_vtctldata_proto_msgTypes[136] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8517,7 +8782,7 @@ func (x *MountListRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MountListRequest.ProtoReflect.Descriptor instead. func (*MountListRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{131} + return file_vtctldata_proto_rawDescGZIP(), []int{136} } type MountListResponse struct { @@ -8531,7 +8796,7 @@ type MountListResponse struct { func (x *MountListResponse) Reset() { *x = MountListResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[132] + mi := &file_vtctldata_proto_msgTypes[137] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8544,7 +8809,7 @@ func (x *MountListResponse) String() string { func (*MountListResponse) ProtoMessage() {} func (x *MountListResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[132] + mi := &file_vtctldata_proto_msgTypes[137] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8557,7 +8822,7 @@ func (x *MountListResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MountListResponse.ProtoReflect.Descriptor instead. func (*MountListResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{132} + return file_vtctldata_proto_rawDescGZIP(), []int{137} } func (x *MountListResponse) GetNames() []string { @@ -8601,13 +8866,14 @@ type MoveTablesCreateRequest struct { // 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"` + AtomicCopy bool `protobuf:"varint,19,opt,name=atomic_copy,json=atomicCopy,proto3" json:"atomic_copy,omitempty"` + WorkflowOptions *WorkflowOptions `protobuf:"bytes,20,opt,name=workflow_options,json=workflowOptions,proto3" json:"workflow_options,omitempty"` } func (x *MoveTablesCreateRequest) Reset() { *x = MoveTablesCreateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[133] + mi := &file_vtctldata_proto_msgTypes[138] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8620,7 +8886,7 @@ func (x *MoveTablesCreateRequest) String() string { func (*MoveTablesCreateRequest) ProtoMessage() {} func (x *MoveTablesCreateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[133] + mi := &file_vtctldata_proto_msgTypes[138] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8633,7 +8899,7 @@ func (x *MoveTablesCreateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MoveTablesCreateRequest.ProtoReflect.Descriptor instead. func (*MoveTablesCreateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{133} + return file_vtctldata_proto_rawDescGZIP(), []int{138} } func (x *MoveTablesCreateRequest) GetWorkflow() string { @@ -8769,6 +9035,13 @@ func (x *MoveTablesCreateRequest) GetAtomicCopy() bool { return false } +func (x *MoveTablesCreateRequest) GetWorkflowOptions() *WorkflowOptions { + if x != nil { + return x.WorkflowOptions + } + return nil +} + type MoveTablesCreateResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -8781,7 +9054,7 @@ type MoveTablesCreateResponse struct { func (x *MoveTablesCreateResponse) Reset() { *x = MoveTablesCreateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[134] + mi := &file_vtctldata_proto_msgTypes[139] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8794,7 +9067,7 @@ func (x *MoveTablesCreateResponse) String() string { func (*MoveTablesCreateResponse) ProtoMessage() {} func (x *MoveTablesCreateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[134] + mi := &file_vtctldata_proto_msgTypes[139] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8807,7 +9080,7 @@ func (x *MoveTablesCreateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MoveTablesCreateResponse.ProtoReflect.Descriptor instead. func (*MoveTablesCreateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{134} + return file_vtctldata_proto_rawDescGZIP(), []int{139} } func (x *MoveTablesCreateResponse) GetSummary() string { @@ -8841,7 +9114,7 @@ type MoveTablesCompleteRequest struct { func (x *MoveTablesCompleteRequest) Reset() { *x = MoveTablesCompleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[135] + mi := &file_vtctldata_proto_msgTypes[140] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8854,7 +9127,7 @@ func (x *MoveTablesCompleteRequest) String() string { func (*MoveTablesCompleteRequest) ProtoMessage() {} func (x *MoveTablesCompleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[135] + mi := &file_vtctldata_proto_msgTypes[140] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8867,7 +9140,7 @@ func (x *MoveTablesCompleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MoveTablesCompleteRequest.ProtoReflect.Descriptor instead. func (*MoveTablesCompleteRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{135} + return file_vtctldata_proto_rawDescGZIP(), []int{140} } func (x *MoveTablesCompleteRequest) GetWorkflow() string { @@ -8931,7 +9204,7 @@ type MoveTablesCompleteResponse struct { func (x *MoveTablesCompleteResponse) Reset() { *x = MoveTablesCompleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[136] + mi := &file_vtctldata_proto_msgTypes[141] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8944,7 +9217,7 @@ func (x *MoveTablesCompleteResponse) String() string { func (*MoveTablesCompleteResponse) ProtoMessage() {} func (x *MoveTablesCompleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[136] + mi := &file_vtctldata_proto_msgTypes[141] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8957,7 +9230,7 @@ func (x *MoveTablesCompleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use MoveTablesCompleteResponse.ProtoReflect.Descriptor instead. func (*MoveTablesCompleteResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{136} + return file_vtctldata_proto_rawDescGZIP(), []int{141} } func (x *MoveTablesCompleteResponse) GetSummary() string { @@ -8985,7 +9258,7 @@ type PingTabletRequest struct { func (x *PingTabletRequest) Reset() { *x = PingTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[137] + mi := &file_vtctldata_proto_msgTypes[142] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8998,7 +9271,7 @@ func (x *PingTabletRequest) String() string { func (*PingTabletRequest) ProtoMessage() {} func (x *PingTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[137] + mi := &file_vtctldata_proto_msgTypes[142] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9011,7 +9284,7 @@ func (x *PingTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PingTabletRequest.ProtoReflect.Descriptor instead. func (*PingTabletRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{137} + return file_vtctldata_proto_rawDescGZIP(), []int{142} } func (x *PingTabletRequest) GetTabletAlias() *topodata.TabletAlias { @@ -9030,7 +9303,7 @@ type PingTabletResponse struct { func (x *PingTabletResponse) Reset() { *x = PingTabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[138] + mi := &file_vtctldata_proto_msgTypes[143] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9043,7 +9316,7 @@ func (x *PingTabletResponse) String() string { func (*PingTabletResponse) ProtoMessage() {} func (x *PingTabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[138] + mi := &file_vtctldata_proto_msgTypes[143] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9056,7 +9329,7 @@ func (x *PingTabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PingTabletResponse.ProtoReflect.Descriptor instead. func (*PingTabletResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{138} + return file_vtctldata_proto_rawDescGZIP(), []int{143} } type PlannedReparentShardRequest struct { @@ -9095,7 +9368,7 @@ type PlannedReparentShardRequest struct { func (x *PlannedReparentShardRequest) Reset() { *x = PlannedReparentShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[139] + mi := &file_vtctldata_proto_msgTypes[144] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9108,7 +9381,7 @@ func (x *PlannedReparentShardRequest) String() string { func (*PlannedReparentShardRequest) ProtoMessage() {} func (x *PlannedReparentShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[139] + mi := &file_vtctldata_proto_msgTypes[144] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9121,7 +9394,7 @@ func (x *PlannedReparentShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PlannedReparentShardRequest.ProtoReflect.Descriptor instead. func (*PlannedReparentShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{139} + return file_vtctldata_proto_rawDescGZIP(), []int{144} } func (x *PlannedReparentShardRequest) GetKeyspace() string { @@ -9186,7 +9459,7 @@ type PlannedReparentShardResponse struct { func (x *PlannedReparentShardResponse) Reset() { *x = PlannedReparentShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[140] + mi := &file_vtctldata_proto_msgTypes[145] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9199,7 +9472,7 @@ func (x *PlannedReparentShardResponse) String() string { func (*PlannedReparentShardResponse) ProtoMessage() {} func (x *PlannedReparentShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[140] + mi := &file_vtctldata_proto_msgTypes[145] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9212,7 +9485,7 @@ func (x *PlannedReparentShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PlannedReparentShardResponse.ProtoReflect.Descriptor instead. func (*PlannedReparentShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{140} + return file_vtctldata_proto_rawDescGZIP(), []int{145} } func (x *PlannedReparentShardResponse) GetKeyspace() string { @@ -9258,7 +9531,7 @@ type RebuildKeyspaceGraphRequest struct { func (x *RebuildKeyspaceGraphRequest) Reset() { *x = RebuildKeyspaceGraphRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[141] + mi := &file_vtctldata_proto_msgTypes[146] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9271,7 +9544,7 @@ func (x *RebuildKeyspaceGraphRequest) String() string { func (*RebuildKeyspaceGraphRequest) ProtoMessage() {} func (x *RebuildKeyspaceGraphRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[141] + mi := &file_vtctldata_proto_msgTypes[146] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9284,7 +9557,7 @@ func (x *RebuildKeyspaceGraphRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RebuildKeyspaceGraphRequest.ProtoReflect.Descriptor instead. func (*RebuildKeyspaceGraphRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{141} + return file_vtctldata_proto_rawDescGZIP(), []int{146} } func (x *RebuildKeyspaceGraphRequest) GetKeyspace() string { @@ -9317,7 +9590,7 @@ type RebuildKeyspaceGraphResponse struct { func (x *RebuildKeyspaceGraphResponse) Reset() { *x = RebuildKeyspaceGraphResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[142] + mi := &file_vtctldata_proto_msgTypes[147] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9330,7 +9603,7 @@ func (x *RebuildKeyspaceGraphResponse) String() string { func (*RebuildKeyspaceGraphResponse) ProtoMessage() {} func (x *RebuildKeyspaceGraphResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[142] + mi := &file_vtctldata_proto_msgTypes[147] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9343,7 +9616,7 @@ func (x *RebuildKeyspaceGraphResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RebuildKeyspaceGraphResponse.ProtoReflect.Descriptor instead. func (*RebuildKeyspaceGraphResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{142} + return file_vtctldata_proto_rawDescGZIP(), []int{147} } type RebuildVSchemaGraphRequest struct { @@ -9359,7 +9632,7 @@ type RebuildVSchemaGraphRequest struct { func (x *RebuildVSchemaGraphRequest) Reset() { *x = RebuildVSchemaGraphRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[143] + mi := &file_vtctldata_proto_msgTypes[148] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9372,7 +9645,7 @@ func (x *RebuildVSchemaGraphRequest) String() string { func (*RebuildVSchemaGraphRequest) ProtoMessage() {} func (x *RebuildVSchemaGraphRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[143] + mi := &file_vtctldata_proto_msgTypes[148] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9385,7 +9658,7 @@ func (x *RebuildVSchemaGraphRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RebuildVSchemaGraphRequest.ProtoReflect.Descriptor instead. func (*RebuildVSchemaGraphRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{143} + return file_vtctldata_proto_rawDescGZIP(), []int{148} } func (x *RebuildVSchemaGraphRequest) GetCells() []string { @@ -9404,7 +9677,7 @@ type RebuildVSchemaGraphResponse struct { func (x *RebuildVSchemaGraphResponse) Reset() { *x = RebuildVSchemaGraphResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[144] + mi := &file_vtctldata_proto_msgTypes[149] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9417,7 +9690,7 @@ func (x *RebuildVSchemaGraphResponse) String() string { func (*RebuildVSchemaGraphResponse) ProtoMessage() {} func (x *RebuildVSchemaGraphResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[144] + mi := &file_vtctldata_proto_msgTypes[149] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9430,7 +9703,7 @@ func (x *RebuildVSchemaGraphResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RebuildVSchemaGraphResponse.ProtoReflect.Descriptor instead. func (*RebuildVSchemaGraphResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{144} + return file_vtctldata_proto_rawDescGZIP(), []int{149} } type RefreshStateRequest struct { @@ -9444,7 +9717,7 @@ type RefreshStateRequest struct { func (x *RefreshStateRequest) Reset() { *x = RefreshStateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[145] + mi := &file_vtctldata_proto_msgTypes[150] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9457,7 +9730,7 @@ func (x *RefreshStateRequest) String() string { func (*RefreshStateRequest) ProtoMessage() {} func (x *RefreshStateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[145] + mi := &file_vtctldata_proto_msgTypes[150] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9470,7 +9743,7 @@ func (x *RefreshStateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RefreshStateRequest.ProtoReflect.Descriptor instead. func (*RefreshStateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{145} + return file_vtctldata_proto_rawDescGZIP(), []int{150} } func (x *RefreshStateRequest) GetTabletAlias() *topodata.TabletAlias { @@ -9489,7 +9762,7 @@ type RefreshStateResponse struct { func (x *RefreshStateResponse) Reset() { *x = RefreshStateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[146] + mi := &file_vtctldata_proto_msgTypes[151] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9502,7 +9775,7 @@ func (x *RefreshStateResponse) String() string { func (*RefreshStateResponse) ProtoMessage() {} func (x *RefreshStateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[146] + mi := &file_vtctldata_proto_msgTypes[151] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9515,7 +9788,7 @@ func (x *RefreshStateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RefreshStateResponse.ProtoReflect.Descriptor instead. func (*RefreshStateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{146} + return file_vtctldata_proto_rawDescGZIP(), []int{151} } type RefreshStateByShardRequest struct { @@ -9531,7 +9804,7 @@ type RefreshStateByShardRequest struct { func (x *RefreshStateByShardRequest) Reset() { *x = RefreshStateByShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[147] + mi := &file_vtctldata_proto_msgTypes[152] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9544,7 +9817,7 @@ func (x *RefreshStateByShardRequest) String() string { func (*RefreshStateByShardRequest) ProtoMessage() {} func (x *RefreshStateByShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[147] + mi := &file_vtctldata_proto_msgTypes[152] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9557,7 +9830,7 @@ func (x *RefreshStateByShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RefreshStateByShardRequest.ProtoReflect.Descriptor instead. func (*RefreshStateByShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{147} + return file_vtctldata_proto_rawDescGZIP(), []int{152} } func (x *RefreshStateByShardRequest) GetKeyspace() string { @@ -9594,7 +9867,7 @@ type RefreshStateByShardResponse struct { func (x *RefreshStateByShardResponse) Reset() { *x = RefreshStateByShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[148] + mi := &file_vtctldata_proto_msgTypes[153] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9607,7 +9880,7 @@ func (x *RefreshStateByShardResponse) String() string { func (*RefreshStateByShardResponse) ProtoMessage() {} func (x *RefreshStateByShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[148] + mi := &file_vtctldata_proto_msgTypes[153] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9620,7 +9893,7 @@ func (x *RefreshStateByShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RefreshStateByShardResponse.ProtoReflect.Descriptor instead. func (*RefreshStateByShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{148} + return file_vtctldata_proto_rawDescGZIP(), []int{153} } func (x *RefreshStateByShardResponse) GetIsPartialRefresh() bool { @@ -9648,7 +9921,7 @@ type ReloadSchemaRequest struct { func (x *ReloadSchemaRequest) Reset() { *x = ReloadSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[149] + mi := &file_vtctldata_proto_msgTypes[154] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9661,7 +9934,7 @@ func (x *ReloadSchemaRequest) String() string { func (*ReloadSchemaRequest) ProtoMessage() {} func (x *ReloadSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[149] + mi := &file_vtctldata_proto_msgTypes[154] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9674,7 +9947,7 @@ func (x *ReloadSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaRequest.ProtoReflect.Descriptor instead. func (*ReloadSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{149} + return file_vtctldata_proto_rawDescGZIP(), []int{154} } func (x *ReloadSchemaRequest) GetTabletAlias() *topodata.TabletAlias { @@ -9693,7 +9966,7 @@ type ReloadSchemaResponse struct { func (x *ReloadSchemaResponse) Reset() { *x = ReloadSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[150] + mi := &file_vtctldata_proto_msgTypes[155] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9706,7 +9979,7 @@ func (x *ReloadSchemaResponse) String() string { func (*ReloadSchemaResponse) ProtoMessage() {} func (x *ReloadSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[150] + mi := &file_vtctldata_proto_msgTypes[155] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9719,7 +9992,7 @@ func (x *ReloadSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaResponse.ProtoReflect.Descriptor instead. func (*ReloadSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{150} + return file_vtctldata_proto_rawDescGZIP(), []int{155} } type ReloadSchemaKeyspaceRequest struct { @@ -9739,7 +10012,7 @@ type ReloadSchemaKeyspaceRequest struct { func (x *ReloadSchemaKeyspaceRequest) Reset() { *x = ReloadSchemaKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[151] + mi := &file_vtctldata_proto_msgTypes[156] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9752,7 +10025,7 @@ func (x *ReloadSchemaKeyspaceRequest) String() string { func (*ReloadSchemaKeyspaceRequest) ProtoMessage() {} func (x *ReloadSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[151] + mi := &file_vtctldata_proto_msgTypes[156] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9765,7 +10038,7 @@ func (x *ReloadSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ReloadSchemaKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{151} + return file_vtctldata_proto_rawDescGZIP(), []int{156} } func (x *ReloadSchemaKeyspaceRequest) GetKeyspace() string { @@ -9807,7 +10080,7 @@ type ReloadSchemaKeyspaceResponse struct { func (x *ReloadSchemaKeyspaceResponse) Reset() { *x = ReloadSchemaKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[152] + mi := &file_vtctldata_proto_msgTypes[157] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9820,7 +10093,7 @@ func (x *ReloadSchemaKeyspaceResponse) String() string { func (*ReloadSchemaKeyspaceResponse) ProtoMessage() {} func (x *ReloadSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[152] + mi := &file_vtctldata_proto_msgTypes[157] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9833,7 +10106,7 @@ func (x *ReloadSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ReloadSchemaKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{152} + return file_vtctldata_proto_rawDescGZIP(), []int{157} } func (x *ReloadSchemaKeyspaceResponse) GetEvents() []*logutil.Event { @@ -9859,7 +10132,7 @@ type ReloadSchemaShardRequest struct { func (x *ReloadSchemaShardRequest) Reset() { *x = ReloadSchemaShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[153] + mi := &file_vtctldata_proto_msgTypes[158] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9872,7 +10145,7 @@ func (x *ReloadSchemaShardRequest) String() string { func (*ReloadSchemaShardRequest) ProtoMessage() {} func (x *ReloadSchemaShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[153] + mi := &file_vtctldata_proto_msgTypes[158] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9885,7 +10158,7 @@ func (x *ReloadSchemaShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaShardRequest.ProtoReflect.Descriptor instead. func (*ReloadSchemaShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{153} + return file_vtctldata_proto_rawDescGZIP(), []int{158} } func (x *ReloadSchemaShardRequest) GetKeyspace() string { @@ -9934,7 +10207,7 @@ type ReloadSchemaShardResponse struct { func (x *ReloadSchemaShardResponse) Reset() { *x = ReloadSchemaShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[154] + mi := &file_vtctldata_proto_msgTypes[159] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9947,7 +10220,7 @@ func (x *ReloadSchemaShardResponse) String() string { func (*ReloadSchemaShardResponse) ProtoMessage() {} func (x *ReloadSchemaShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[154] + mi := &file_vtctldata_proto_msgTypes[159] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9960,7 +10233,7 @@ func (x *ReloadSchemaShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReloadSchemaShardResponse.ProtoReflect.Descriptor instead. func (*ReloadSchemaShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{154} + return file_vtctldata_proto_rawDescGZIP(), []int{159} } func (x *ReloadSchemaShardResponse) GetEvents() []*logutil.Event { @@ -9983,7 +10256,7 @@ type RemoveBackupRequest struct { func (x *RemoveBackupRequest) Reset() { *x = RemoveBackupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[155] + mi := &file_vtctldata_proto_msgTypes[160] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9996,7 +10269,7 @@ func (x *RemoveBackupRequest) String() string { func (*RemoveBackupRequest) ProtoMessage() {} func (x *RemoveBackupRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[155] + mi := &file_vtctldata_proto_msgTypes[160] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10009,7 +10282,7 @@ func (x *RemoveBackupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveBackupRequest.ProtoReflect.Descriptor instead. func (*RemoveBackupRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{155} + return file_vtctldata_proto_rawDescGZIP(), []int{160} } func (x *RemoveBackupRequest) GetKeyspace() string { @@ -10042,7 +10315,7 @@ type RemoveBackupResponse struct { func (x *RemoveBackupResponse) Reset() { *x = RemoveBackupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[156] + mi := &file_vtctldata_proto_msgTypes[161] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10055,7 +10328,7 @@ func (x *RemoveBackupResponse) String() string { func (*RemoveBackupResponse) ProtoMessage() {} func (x *RemoveBackupResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[156] + mi := &file_vtctldata_proto_msgTypes[161] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10068,7 +10341,7 @@ func (x *RemoveBackupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveBackupResponse.ProtoReflect.Descriptor instead. func (*RemoveBackupResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{156} + return file_vtctldata_proto_rawDescGZIP(), []int{161} } type RemoveKeyspaceCellRequest struct { @@ -10090,7 +10363,7 @@ type RemoveKeyspaceCellRequest struct { func (x *RemoveKeyspaceCellRequest) Reset() { *x = RemoveKeyspaceCellRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[157] + mi := &file_vtctldata_proto_msgTypes[162] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10103,7 +10376,7 @@ func (x *RemoveKeyspaceCellRequest) String() string { func (*RemoveKeyspaceCellRequest) ProtoMessage() {} func (x *RemoveKeyspaceCellRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[157] + mi := &file_vtctldata_proto_msgTypes[162] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10116,7 +10389,7 @@ func (x *RemoveKeyspaceCellRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveKeyspaceCellRequest.ProtoReflect.Descriptor instead. func (*RemoveKeyspaceCellRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{157} + return file_vtctldata_proto_rawDescGZIP(), []int{162} } func (x *RemoveKeyspaceCellRequest) GetKeyspace() string { @@ -10156,7 +10429,7 @@ type RemoveKeyspaceCellResponse struct { func (x *RemoveKeyspaceCellResponse) Reset() { *x = RemoveKeyspaceCellResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[158] + mi := &file_vtctldata_proto_msgTypes[163] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10169,7 +10442,7 @@ func (x *RemoveKeyspaceCellResponse) String() string { func (*RemoveKeyspaceCellResponse) ProtoMessage() {} func (x *RemoveKeyspaceCellResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[158] + mi := &file_vtctldata_proto_msgTypes[163] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10182,7 +10455,7 @@ func (x *RemoveKeyspaceCellResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveKeyspaceCellResponse.ProtoReflect.Descriptor instead. func (*RemoveKeyspaceCellResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{158} + return file_vtctldata_proto_rawDescGZIP(), []int{163} } type RemoveShardCellRequest struct { @@ -10205,7 +10478,7 @@ type RemoveShardCellRequest struct { func (x *RemoveShardCellRequest) Reset() { *x = RemoveShardCellRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[159] + mi := &file_vtctldata_proto_msgTypes[164] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10218,7 +10491,7 @@ func (x *RemoveShardCellRequest) String() string { func (*RemoveShardCellRequest) ProtoMessage() {} func (x *RemoveShardCellRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[159] + mi := &file_vtctldata_proto_msgTypes[164] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10231,7 +10504,7 @@ func (x *RemoveShardCellRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveShardCellRequest.ProtoReflect.Descriptor instead. func (*RemoveShardCellRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{159} + return file_vtctldata_proto_rawDescGZIP(), []int{164} } func (x *RemoveShardCellRequest) GetKeyspace() string { @@ -10278,7 +10551,7 @@ type RemoveShardCellResponse struct { func (x *RemoveShardCellResponse) Reset() { *x = RemoveShardCellResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[160] + mi := &file_vtctldata_proto_msgTypes[165] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10291,7 +10564,7 @@ func (x *RemoveShardCellResponse) String() string { func (*RemoveShardCellResponse) ProtoMessage() {} func (x *RemoveShardCellResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[160] + mi := &file_vtctldata_proto_msgTypes[165] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10304,7 +10577,7 @@ func (x *RemoveShardCellResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RemoveShardCellResponse.ProtoReflect.Descriptor instead. func (*RemoveShardCellResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{160} + return file_vtctldata_proto_rawDescGZIP(), []int{165} } type ReparentTabletRequest struct { @@ -10320,7 +10593,7 @@ type ReparentTabletRequest struct { func (x *ReparentTabletRequest) Reset() { *x = ReparentTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[161] + mi := &file_vtctldata_proto_msgTypes[166] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10333,7 +10606,7 @@ func (x *ReparentTabletRequest) String() string { func (*ReparentTabletRequest) ProtoMessage() {} func (x *ReparentTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[161] + mi := &file_vtctldata_proto_msgTypes[166] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10346,7 +10619,7 @@ func (x *ReparentTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReparentTabletRequest.ProtoReflect.Descriptor instead. func (*ReparentTabletRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{161} + return file_vtctldata_proto_rawDescGZIP(), []int{166} } func (x *ReparentTabletRequest) GetTablet() *topodata.TabletAlias { @@ -10372,7 +10645,7 @@ type ReparentTabletResponse struct { func (x *ReparentTabletResponse) Reset() { *x = ReparentTabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[162] + mi := &file_vtctldata_proto_msgTypes[167] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10385,7 +10658,7 @@ func (x *ReparentTabletResponse) String() string { func (*ReparentTabletResponse) ProtoMessage() {} func (x *ReparentTabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[162] + mi := &file_vtctldata_proto_msgTypes[167] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10398,7 +10671,7 @@ func (x *ReparentTabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReparentTabletResponse.ProtoReflect.Descriptor instead. func (*ReparentTabletResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{162} + return file_vtctldata_proto_rawDescGZIP(), []int{167} } func (x *ReparentTabletResponse) GetKeyspace() string { @@ -10450,7 +10723,7 @@ type ReshardCreateRequest struct { func (x *ReshardCreateRequest) Reset() { *x = ReshardCreateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[163] + mi := &file_vtctldata_proto_msgTypes[168] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10463,7 +10736,7 @@ func (x *ReshardCreateRequest) String() string { func (*ReshardCreateRequest) ProtoMessage() {} func (x *ReshardCreateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[163] + mi := &file_vtctldata_proto_msgTypes[168] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10476,7 +10749,7 @@ func (x *ReshardCreateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReshardCreateRequest.ProtoReflect.Descriptor instead. func (*ReshardCreateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{163} + return file_vtctldata_proto_rawDescGZIP(), []int{168} } func (x *ReshardCreateRequest) GetWorkflow() string { @@ -10586,7 +10859,7 @@ type RestoreFromBackupRequest struct { func (x *RestoreFromBackupRequest) Reset() { *x = RestoreFromBackupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[164] + mi := &file_vtctldata_proto_msgTypes[169] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10599,7 +10872,7 @@ func (x *RestoreFromBackupRequest) String() string { func (*RestoreFromBackupRequest) ProtoMessage() {} func (x *RestoreFromBackupRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[164] + mi := &file_vtctldata_proto_msgTypes[169] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10612,7 +10885,7 @@ func (x *RestoreFromBackupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RestoreFromBackupRequest.ProtoReflect.Descriptor instead. func (*RestoreFromBackupRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{164} + return file_vtctldata_proto_rawDescGZIP(), []int{169} } func (x *RestoreFromBackupRequest) GetTabletAlias() *topodata.TabletAlias { @@ -10665,7 +10938,7 @@ type RestoreFromBackupResponse struct { func (x *RestoreFromBackupResponse) Reset() { *x = RestoreFromBackupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[165] + mi := &file_vtctldata_proto_msgTypes[170] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10678,7 +10951,7 @@ func (x *RestoreFromBackupResponse) String() string { func (*RestoreFromBackupResponse) ProtoMessage() {} func (x *RestoreFromBackupResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[165] + mi := &file_vtctldata_proto_msgTypes[170] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10691,7 +10964,7 @@ func (x *RestoreFromBackupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RestoreFromBackupResponse.ProtoReflect.Descriptor instead. func (*RestoreFromBackupResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{165} + return file_vtctldata_proto_rawDescGZIP(), []int{170} } func (x *RestoreFromBackupResponse) GetTabletAlias() *topodata.TabletAlias { @@ -10734,7 +11007,7 @@ type RetrySchemaMigrationRequest struct { func (x *RetrySchemaMigrationRequest) Reset() { *x = RetrySchemaMigrationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[166] + mi := &file_vtctldata_proto_msgTypes[171] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10747,7 +11020,7 @@ func (x *RetrySchemaMigrationRequest) String() string { func (*RetrySchemaMigrationRequest) ProtoMessage() {} func (x *RetrySchemaMigrationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[166] + mi := &file_vtctldata_proto_msgTypes[171] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10760,7 +11033,7 @@ func (x *RetrySchemaMigrationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RetrySchemaMigrationRequest.ProtoReflect.Descriptor instead. func (*RetrySchemaMigrationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{166} + return file_vtctldata_proto_rawDescGZIP(), []int{171} } func (x *RetrySchemaMigrationRequest) GetKeyspace() string { @@ -10788,7 +11061,7 @@ type RetrySchemaMigrationResponse struct { func (x *RetrySchemaMigrationResponse) Reset() { *x = RetrySchemaMigrationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[167] + mi := &file_vtctldata_proto_msgTypes[172] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10801,7 +11074,7 @@ func (x *RetrySchemaMigrationResponse) String() string { func (*RetrySchemaMigrationResponse) ProtoMessage() {} func (x *RetrySchemaMigrationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[167] + mi := &file_vtctldata_proto_msgTypes[172] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10814,7 +11087,7 @@ func (x *RetrySchemaMigrationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RetrySchemaMigrationResponse.ProtoReflect.Descriptor instead. func (*RetrySchemaMigrationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{167} + return file_vtctldata_proto_rawDescGZIP(), []int{172} } func (x *RetrySchemaMigrationResponse) GetRowsAffectedByShard() map[string]uint64 { @@ -10835,7 +11108,7 @@ type RunHealthCheckRequest struct { func (x *RunHealthCheckRequest) Reset() { *x = RunHealthCheckRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[168] + mi := &file_vtctldata_proto_msgTypes[173] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10848,7 +11121,7 @@ func (x *RunHealthCheckRequest) String() string { func (*RunHealthCheckRequest) ProtoMessage() {} func (x *RunHealthCheckRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[168] + mi := &file_vtctldata_proto_msgTypes[173] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10861,7 +11134,7 @@ func (x *RunHealthCheckRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RunHealthCheckRequest.ProtoReflect.Descriptor instead. func (*RunHealthCheckRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{168} + return file_vtctldata_proto_rawDescGZIP(), []int{173} } func (x *RunHealthCheckRequest) GetTabletAlias() *topodata.TabletAlias { @@ -10880,7 +11153,7 @@ type RunHealthCheckResponse struct { func (x *RunHealthCheckResponse) Reset() { *x = RunHealthCheckResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[169] + mi := &file_vtctldata_proto_msgTypes[174] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10893,7 +11166,7 @@ func (x *RunHealthCheckResponse) String() string { func (*RunHealthCheckResponse) ProtoMessage() {} func (x *RunHealthCheckResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[169] + mi := &file_vtctldata_proto_msgTypes[174] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10906,7 +11179,7 @@ func (x *RunHealthCheckResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RunHealthCheckResponse.ProtoReflect.Descriptor instead. func (*RunHealthCheckResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{169} + return file_vtctldata_proto_rawDescGZIP(), []int{174} } type SetKeyspaceDurabilityPolicyRequest struct { @@ -10921,7 +11194,7 @@ type SetKeyspaceDurabilityPolicyRequest struct { func (x *SetKeyspaceDurabilityPolicyRequest) Reset() { *x = SetKeyspaceDurabilityPolicyRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[170] + mi := &file_vtctldata_proto_msgTypes[175] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10934,7 +11207,7 @@ func (x *SetKeyspaceDurabilityPolicyRequest) String() string { func (*SetKeyspaceDurabilityPolicyRequest) ProtoMessage() {} func (x *SetKeyspaceDurabilityPolicyRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[170] + mi := &file_vtctldata_proto_msgTypes[175] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -10947,7 +11220,7 @@ func (x *SetKeyspaceDurabilityPolicyRequest) ProtoReflect() protoreflect.Message // Deprecated: Use SetKeyspaceDurabilityPolicyRequest.ProtoReflect.Descriptor instead. func (*SetKeyspaceDurabilityPolicyRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{170} + return file_vtctldata_proto_rawDescGZIP(), []int{175} } func (x *SetKeyspaceDurabilityPolicyRequest) GetKeyspace() string { @@ -10976,7 +11249,7 @@ type SetKeyspaceDurabilityPolicyResponse struct { func (x *SetKeyspaceDurabilityPolicyResponse) Reset() { *x = SetKeyspaceDurabilityPolicyResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[171] + mi := &file_vtctldata_proto_msgTypes[176] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -10989,7 +11262,7 @@ func (x *SetKeyspaceDurabilityPolicyResponse) String() string { func (*SetKeyspaceDurabilityPolicyResponse) ProtoMessage() {} func (x *SetKeyspaceDurabilityPolicyResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[171] + mi := &file_vtctldata_proto_msgTypes[176] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11002,7 +11275,7 @@ func (x *SetKeyspaceDurabilityPolicyResponse) ProtoReflect() protoreflect.Messag // Deprecated: Use SetKeyspaceDurabilityPolicyResponse.ProtoReflect.Descriptor instead. func (*SetKeyspaceDurabilityPolicyResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{171} + return file_vtctldata_proto_rawDescGZIP(), []int{176} } func (x *SetKeyspaceDurabilityPolicyResponse) GetKeyspace() *topodata.Keyspace { @@ -11024,7 +11297,7 @@ type SetKeyspaceShardingInfoRequest struct { func (x *SetKeyspaceShardingInfoRequest) Reset() { *x = SetKeyspaceShardingInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[172] + mi := &file_vtctldata_proto_msgTypes[177] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11037,7 +11310,7 @@ func (x *SetKeyspaceShardingInfoRequest) String() string { func (*SetKeyspaceShardingInfoRequest) ProtoMessage() {} func (x *SetKeyspaceShardingInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[172] + mi := &file_vtctldata_proto_msgTypes[177] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11050,7 +11323,7 @@ func (x *SetKeyspaceShardingInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetKeyspaceShardingInfoRequest.ProtoReflect.Descriptor instead. func (*SetKeyspaceShardingInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{172} + return file_vtctldata_proto_rawDescGZIP(), []int{177} } func (x *SetKeyspaceShardingInfoRequest) GetKeyspace() string { @@ -11079,7 +11352,7 @@ type SetKeyspaceShardingInfoResponse struct { func (x *SetKeyspaceShardingInfoResponse) Reset() { *x = SetKeyspaceShardingInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[173] + mi := &file_vtctldata_proto_msgTypes[178] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11092,7 +11365,7 @@ func (x *SetKeyspaceShardingInfoResponse) String() string { func (*SetKeyspaceShardingInfoResponse) ProtoMessage() {} func (x *SetKeyspaceShardingInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[173] + mi := &file_vtctldata_proto_msgTypes[178] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11105,7 +11378,7 @@ func (x *SetKeyspaceShardingInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetKeyspaceShardingInfoResponse.ProtoReflect.Descriptor instead. func (*SetKeyspaceShardingInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{173} + return file_vtctldata_proto_rawDescGZIP(), []int{178} } func (x *SetKeyspaceShardingInfoResponse) GetKeyspace() *topodata.Keyspace { @@ -11128,7 +11401,7 @@ type SetShardIsPrimaryServingRequest struct { func (x *SetShardIsPrimaryServingRequest) Reset() { *x = SetShardIsPrimaryServingRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[174] + mi := &file_vtctldata_proto_msgTypes[179] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11141,7 +11414,7 @@ func (x *SetShardIsPrimaryServingRequest) String() string { func (*SetShardIsPrimaryServingRequest) ProtoMessage() {} func (x *SetShardIsPrimaryServingRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[174] + mi := &file_vtctldata_proto_msgTypes[179] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11154,7 +11427,7 @@ func (x *SetShardIsPrimaryServingRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardIsPrimaryServingRequest.ProtoReflect.Descriptor instead. func (*SetShardIsPrimaryServingRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{174} + return file_vtctldata_proto_rawDescGZIP(), []int{179} } func (x *SetShardIsPrimaryServingRequest) GetKeyspace() string { @@ -11190,7 +11463,7 @@ type SetShardIsPrimaryServingResponse struct { func (x *SetShardIsPrimaryServingResponse) Reset() { *x = SetShardIsPrimaryServingResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[175] + mi := &file_vtctldata_proto_msgTypes[180] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11203,7 +11476,7 @@ func (x *SetShardIsPrimaryServingResponse) String() string { func (*SetShardIsPrimaryServingResponse) ProtoMessage() {} func (x *SetShardIsPrimaryServingResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[175] + mi := &file_vtctldata_proto_msgTypes[180] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11216,7 +11489,7 @@ func (x *SetShardIsPrimaryServingResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardIsPrimaryServingResponse.ProtoReflect.Descriptor instead. func (*SetShardIsPrimaryServingResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{175} + return file_vtctldata_proto_rawDescGZIP(), []int{180} } func (x *SetShardIsPrimaryServingResponse) GetShard() *topodata.Shard { @@ -11257,7 +11530,7 @@ type SetShardTabletControlRequest struct { func (x *SetShardTabletControlRequest) Reset() { *x = SetShardTabletControlRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[176] + mi := &file_vtctldata_proto_msgTypes[181] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11270,7 +11543,7 @@ func (x *SetShardTabletControlRequest) String() string { func (*SetShardTabletControlRequest) ProtoMessage() {} func (x *SetShardTabletControlRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[176] + mi := &file_vtctldata_proto_msgTypes[181] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11283,7 +11556,7 @@ func (x *SetShardTabletControlRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardTabletControlRequest.ProtoReflect.Descriptor instead. func (*SetShardTabletControlRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{176} + return file_vtctldata_proto_rawDescGZIP(), []int{181} } func (x *SetShardTabletControlRequest) GetKeyspace() string { @@ -11347,7 +11620,7 @@ type SetShardTabletControlResponse struct { func (x *SetShardTabletControlResponse) Reset() { *x = SetShardTabletControlResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[177] + mi := &file_vtctldata_proto_msgTypes[182] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11360,7 +11633,7 @@ func (x *SetShardTabletControlResponse) String() string { func (*SetShardTabletControlResponse) ProtoMessage() {} func (x *SetShardTabletControlResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[177] + mi := &file_vtctldata_proto_msgTypes[182] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11373,7 +11646,7 @@ func (x *SetShardTabletControlResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetShardTabletControlResponse.ProtoReflect.Descriptor instead. func (*SetShardTabletControlResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{177} + return file_vtctldata_proto_rawDescGZIP(), []int{182} } func (x *SetShardTabletControlResponse) GetShard() *topodata.Shard { @@ -11395,7 +11668,7 @@ type SetWritableRequest struct { func (x *SetWritableRequest) Reset() { *x = SetWritableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[178] + mi := &file_vtctldata_proto_msgTypes[183] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11408,7 +11681,7 @@ func (x *SetWritableRequest) String() string { func (*SetWritableRequest) ProtoMessage() {} func (x *SetWritableRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[178] + mi := &file_vtctldata_proto_msgTypes[183] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11421,7 +11694,7 @@ func (x *SetWritableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SetWritableRequest.ProtoReflect.Descriptor instead. func (*SetWritableRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{178} + return file_vtctldata_proto_rawDescGZIP(), []int{183} } func (x *SetWritableRequest) GetTabletAlias() *topodata.TabletAlias { @@ -11447,7 +11720,7 @@ type SetWritableResponse struct { func (x *SetWritableResponse) Reset() { *x = SetWritableResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[179] + mi := &file_vtctldata_proto_msgTypes[184] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11460,7 +11733,7 @@ func (x *SetWritableResponse) String() string { func (*SetWritableResponse) ProtoMessage() {} func (x *SetWritableResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[179] + mi := &file_vtctldata_proto_msgTypes[184] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11473,7 +11746,7 @@ func (x *SetWritableResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SetWritableResponse.ProtoReflect.Descriptor instead. func (*SetWritableResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{179} + return file_vtctldata_proto_rawDescGZIP(), []int{184} } type ShardReplicationAddRequest struct { @@ -11489,7 +11762,7 @@ type ShardReplicationAddRequest struct { func (x *ShardReplicationAddRequest) Reset() { *x = ShardReplicationAddRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[180] + mi := &file_vtctldata_proto_msgTypes[185] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11502,7 +11775,7 @@ func (x *ShardReplicationAddRequest) String() string { func (*ShardReplicationAddRequest) ProtoMessage() {} func (x *ShardReplicationAddRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[180] + mi := &file_vtctldata_proto_msgTypes[185] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11515,7 +11788,7 @@ func (x *ShardReplicationAddRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationAddRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationAddRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{180} + return file_vtctldata_proto_rawDescGZIP(), []int{185} } func (x *ShardReplicationAddRequest) GetKeyspace() string { @@ -11548,7 +11821,7 @@ type ShardReplicationAddResponse struct { func (x *ShardReplicationAddResponse) Reset() { *x = ShardReplicationAddResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[181] + mi := &file_vtctldata_proto_msgTypes[186] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11561,7 +11834,7 @@ func (x *ShardReplicationAddResponse) String() string { func (*ShardReplicationAddResponse) ProtoMessage() {} func (x *ShardReplicationAddResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[181] + mi := &file_vtctldata_proto_msgTypes[186] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11574,7 +11847,7 @@ func (x *ShardReplicationAddResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationAddResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationAddResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{181} + return file_vtctldata_proto_rawDescGZIP(), []int{186} } type ShardReplicationFixRequest struct { @@ -11590,7 +11863,7 @@ type ShardReplicationFixRequest struct { func (x *ShardReplicationFixRequest) Reset() { *x = ShardReplicationFixRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[182] + mi := &file_vtctldata_proto_msgTypes[187] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11603,7 +11876,7 @@ func (x *ShardReplicationFixRequest) String() string { func (*ShardReplicationFixRequest) ProtoMessage() {} func (x *ShardReplicationFixRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[182] + mi := &file_vtctldata_proto_msgTypes[187] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11616,7 +11889,7 @@ func (x *ShardReplicationFixRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationFixRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationFixRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{182} + return file_vtctldata_proto_rawDescGZIP(), []int{187} } func (x *ShardReplicationFixRequest) GetKeyspace() string { @@ -11654,7 +11927,7 @@ type ShardReplicationFixResponse struct { func (x *ShardReplicationFixResponse) Reset() { *x = ShardReplicationFixResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[183] + mi := &file_vtctldata_proto_msgTypes[188] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11667,7 +11940,7 @@ func (x *ShardReplicationFixResponse) String() string { func (*ShardReplicationFixResponse) ProtoMessage() {} func (x *ShardReplicationFixResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[183] + mi := &file_vtctldata_proto_msgTypes[188] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11680,7 +11953,7 @@ func (x *ShardReplicationFixResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationFixResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationFixResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{183} + return file_vtctldata_proto_rawDescGZIP(), []int{188} } func (x *ShardReplicationFixResponse) GetError() *topodata.ShardReplicationError { @@ -11702,7 +11975,7 @@ type ShardReplicationPositionsRequest struct { func (x *ShardReplicationPositionsRequest) Reset() { *x = ShardReplicationPositionsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[184] + mi := &file_vtctldata_proto_msgTypes[189] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11715,7 +11988,7 @@ func (x *ShardReplicationPositionsRequest) String() string { func (*ShardReplicationPositionsRequest) ProtoMessage() {} func (x *ShardReplicationPositionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[184] + mi := &file_vtctldata_proto_msgTypes[189] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11728,7 +12001,7 @@ func (x *ShardReplicationPositionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationPositionsRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationPositionsRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{184} + return file_vtctldata_proto_rawDescGZIP(), []int{189} } func (x *ShardReplicationPositionsRequest) GetKeyspace() string { @@ -11761,7 +12034,7 @@ type ShardReplicationPositionsResponse struct { func (x *ShardReplicationPositionsResponse) Reset() { *x = ShardReplicationPositionsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[185] + mi := &file_vtctldata_proto_msgTypes[190] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11774,7 +12047,7 @@ func (x *ShardReplicationPositionsResponse) String() string { func (*ShardReplicationPositionsResponse) ProtoMessage() {} func (x *ShardReplicationPositionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[185] + mi := &file_vtctldata_proto_msgTypes[190] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11787,7 +12060,7 @@ func (x *ShardReplicationPositionsResponse) ProtoReflect() protoreflect.Message // Deprecated: Use ShardReplicationPositionsResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationPositionsResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{185} + return file_vtctldata_proto_rawDescGZIP(), []int{190} } func (x *ShardReplicationPositionsResponse) GetReplicationStatuses() map[string]*replicationdata.Status { @@ -11817,7 +12090,7 @@ type ShardReplicationRemoveRequest struct { func (x *ShardReplicationRemoveRequest) Reset() { *x = ShardReplicationRemoveRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[186] + mi := &file_vtctldata_proto_msgTypes[191] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11830,7 +12103,7 @@ func (x *ShardReplicationRemoveRequest) String() string { func (*ShardReplicationRemoveRequest) ProtoMessage() {} func (x *ShardReplicationRemoveRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[186] + mi := &file_vtctldata_proto_msgTypes[191] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11843,7 +12116,7 @@ func (x *ShardReplicationRemoveRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationRemoveRequest.ProtoReflect.Descriptor instead. func (*ShardReplicationRemoveRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{186} + return file_vtctldata_proto_rawDescGZIP(), []int{191} } func (x *ShardReplicationRemoveRequest) GetKeyspace() string { @@ -11876,7 +12149,7 @@ type ShardReplicationRemoveResponse struct { func (x *ShardReplicationRemoveResponse) Reset() { *x = ShardReplicationRemoveResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[187] + mi := &file_vtctldata_proto_msgTypes[192] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11889,7 +12162,7 @@ func (x *ShardReplicationRemoveResponse) String() string { func (*ShardReplicationRemoveResponse) ProtoMessage() {} func (x *ShardReplicationRemoveResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[187] + mi := &file_vtctldata_proto_msgTypes[192] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11902,7 +12175,7 @@ func (x *ShardReplicationRemoveResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ShardReplicationRemoveResponse.ProtoReflect.Descriptor instead. func (*ShardReplicationRemoveResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{187} + return file_vtctldata_proto_rawDescGZIP(), []int{192} } type SleepTabletRequest struct { @@ -11917,7 +12190,7 @@ type SleepTabletRequest struct { func (x *SleepTabletRequest) Reset() { *x = SleepTabletRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[188] + mi := &file_vtctldata_proto_msgTypes[193] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11930,7 +12203,7 @@ func (x *SleepTabletRequest) String() string { func (*SleepTabletRequest) ProtoMessage() {} func (x *SleepTabletRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[188] + mi := &file_vtctldata_proto_msgTypes[193] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11943,7 +12216,7 @@ func (x *SleepTabletRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SleepTabletRequest.ProtoReflect.Descriptor instead. func (*SleepTabletRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{188} + return file_vtctldata_proto_rawDescGZIP(), []int{193} } func (x *SleepTabletRequest) GetTabletAlias() *topodata.TabletAlias { @@ -11969,7 +12242,7 @@ type SleepTabletResponse struct { func (x *SleepTabletResponse) Reset() { *x = SleepTabletResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[189] + mi := &file_vtctldata_proto_msgTypes[194] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -11982,7 +12255,7 @@ func (x *SleepTabletResponse) String() string { func (*SleepTabletResponse) ProtoMessage() {} func (x *SleepTabletResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[189] + mi := &file_vtctldata_proto_msgTypes[194] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11995,7 +12268,7 @@ func (x *SleepTabletResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SleepTabletResponse.ProtoReflect.Descriptor instead. func (*SleepTabletResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{189} + return file_vtctldata_proto_rawDescGZIP(), []int{194} } type SourceShardAddRequest struct { @@ -12019,7 +12292,7 @@ type SourceShardAddRequest struct { func (x *SourceShardAddRequest) Reset() { *x = SourceShardAddRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[190] + mi := &file_vtctldata_proto_msgTypes[195] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12032,7 +12305,7 @@ func (x *SourceShardAddRequest) String() string { func (*SourceShardAddRequest) ProtoMessage() {} func (x *SourceShardAddRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[190] + mi := &file_vtctldata_proto_msgTypes[195] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12045,7 +12318,7 @@ func (x *SourceShardAddRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardAddRequest.ProtoReflect.Descriptor instead. func (*SourceShardAddRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{190} + return file_vtctldata_proto_rawDescGZIP(), []int{195} } func (x *SourceShardAddRequest) GetKeyspace() string { @@ -12109,7 +12382,7 @@ type SourceShardAddResponse struct { func (x *SourceShardAddResponse) Reset() { *x = SourceShardAddResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[191] + mi := &file_vtctldata_proto_msgTypes[196] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12122,7 +12395,7 @@ func (x *SourceShardAddResponse) String() string { func (*SourceShardAddResponse) ProtoMessage() {} func (x *SourceShardAddResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[191] + mi := &file_vtctldata_proto_msgTypes[196] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12135,7 +12408,7 @@ func (x *SourceShardAddResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardAddResponse.ProtoReflect.Descriptor instead. func (*SourceShardAddResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{191} + return file_vtctldata_proto_rawDescGZIP(), []int{196} } func (x *SourceShardAddResponse) GetShard() *topodata.Shard { @@ -12158,7 +12431,7 @@ type SourceShardDeleteRequest struct { func (x *SourceShardDeleteRequest) Reset() { *x = SourceShardDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[192] + mi := &file_vtctldata_proto_msgTypes[197] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12171,7 +12444,7 @@ func (x *SourceShardDeleteRequest) String() string { func (*SourceShardDeleteRequest) ProtoMessage() {} func (x *SourceShardDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[192] + mi := &file_vtctldata_proto_msgTypes[197] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12184,7 +12457,7 @@ func (x *SourceShardDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardDeleteRequest.ProtoReflect.Descriptor instead. func (*SourceShardDeleteRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{192} + return file_vtctldata_proto_rawDescGZIP(), []int{197} } func (x *SourceShardDeleteRequest) GetKeyspace() string { @@ -12220,7 +12493,7 @@ type SourceShardDeleteResponse struct { func (x *SourceShardDeleteResponse) Reset() { *x = SourceShardDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[193] + mi := &file_vtctldata_proto_msgTypes[198] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12233,7 +12506,7 @@ func (x *SourceShardDeleteResponse) String() string { func (*SourceShardDeleteResponse) ProtoMessage() {} func (x *SourceShardDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[193] + mi := &file_vtctldata_proto_msgTypes[198] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12246,7 +12519,7 @@ func (x *SourceShardDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SourceShardDeleteResponse.ProtoReflect.Descriptor instead. func (*SourceShardDeleteResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{193} + return file_vtctldata_proto_rawDescGZIP(), []int{198} } func (x *SourceShardDeleteResponse) GetShard() *topodata.Shard { @@ -12267,7 +12540,7 @@ type StartReplicationRequest struct { func (x *StartReplicationRequest) Reset() { *x = StartReplicationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[194] + mi := &file_vtctldata_proto_msgTypes[199] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12280,7 +12553,7 @@ func (x *StartReplicationRequest) String() string { func (*StartReplicationRequest) ProtoMessage() {} func (x *StartReplicationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[194] + mi := &file_vtctldata_proto_msgTypes[199] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12293,7 +12566,7 @@ func (x *StartReplicationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StartReplicationRequest.ProtoReflect.Descriptor instead. func (*StartReplicationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{194} + return file_vtctldata_proto_rawDescGZIP(), []int{199} } func (x *StartReplicationRequest) GetTabletAlias() *topodata.TabletAlias { @@ -12312,7 +12585,7 @@ type StartReplicationResponse struct { func (x *StartReplicationResponse) Reset() { *x = StartReplicationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[195] + mi := &file_vtctldata_proto_msgTypes[200] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12325,7 +12598,7 @@ func (x *StartReplicationResponse) String() string { func (*StartReplicationResponse) ProtoMessage() {} func (x *StartReplicationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[195] + mi := &file_vtctldata_proto_msgTypes[200] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12338,7 +12611,7 @@ func (x *StartReplicationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StartReplicationResponse.ProtoReflect.Descriptor instead. func (*StartReplicationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{195} + return file_vtctldata_proto_rawDescGZIP(), []int{200} } type StopReplicationRequest struct { @@ -12352,7 +12625,7 @@ type StopReplicationRequest struct { func (x *StopReplicationRequest) Reset() { *x = StopReplicationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[196] + mi := &file_vtctldata_proto_msgTypes[201] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12365,7 +12638,7 @@ func (x *StopReplicationRequest) String() string { func (*StopReplicationRequest) ProtoMessage() {} func (x *StopReplicationRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[196] + mi := &file_vtctldata_proto_msgTypes[201] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12378,7 +12651,7 @@ func (x *StopReplicationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use StopReplicationRequest.ProtoReflect.Descriptor instead. func (*StopReplicationRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{196} + return file_vtctldata_proto_rawDescGZIP(), []int{201} } func (x *StopReplicationRequest) GetTabletAlias() *topodata.TabletAlias { @@ -12397,7 +12670,7 @@ type StopReplicationResponse struct { func (x *StopReplicationResponse) Reset() { *x = StopReplicationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[197] + mi := &file_vtctldata_proto_msgTypes[202] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12410,7 +12683,7 @@ func (x *StopReplicationResponse) String() string { func (*StopReplicationResponse) ProtoMessage() {} func (x *StopReplicationResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[197] + mi := &file_vtctldata_proto_msgTypes[202] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12423,7 +12696,7 @@ func (x *StopReplicationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use StopReplicationResponse.ProtoReflect.Descriptor instead. func (*StopReplicationResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{197} + return file_vtctldata_proto_rawDescGZIP(), []int{202} } type TabletExternallyReparentedRequest struct { @@ -12439,7 +12712,7 @@ type TabletExternallyReparentedRequest struct { func (x *TabletExternallyReparentedRequest) Reset() { *x = TabletExternallyReparentedRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[198] + mi := &file_vtctldata_proto_msgTypes[203] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12452,7 +12725,7 @@ func (x *TabletExternallyReparentedRequest) String() string { func (*TabletExternallyReparentedRequest) ProtoMessage() {} func (x *TabletExternallyReparentedRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[198] + mi := &file_vtctldata_proto_msgTypes[203] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12465,7 +12738,7 @@ func (x *TabletExternallyReparentedRequest) ProtoReflect() protoreflect.Message // Deprecated: Use TabletExternallyReparentedRequest.ProtoReflect.Descriptor instead. func (*TabletExternallyReparentedRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{198} + return file_vtctldata_proto_rawDescGZIP(), []int{203} } func (x *TabletExternallyReparentedRequest) GetTablet() *topodata.TabletAlias { @@ -12489,7 +12762,7 @@ type TabletExternallyReparentedResponse struct { func (x *TabletExternallyReparentedResponse) Reset() { *x = TabletExternallyReparentedResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[199] + mi := &file_vtctldata_proto_msgTypes[204] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12502,7 +12775,7 @@ func (x *TabletExternallyReparentedResponse) String() string { func (*TabletExternallyReparentedResponse) ProtoMessage() {} func (x *TabletExternallyReparentedResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[199] + mi := &file_vtctldata_proto_msgTypes[204] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12515,7 +12788,7 @@ func (x *TabletExternallyReparentedResponse) ProtoReflect() protoreflect.Message // Deprecated: Use TabletExternallyReparentedResponse.ProtoReflect.Descriptor instead. func (*TabletExternallyReparentedResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{199} + return file_vtctldata_proto_rawDescGZIP(), []int{204} } func (x *TabletExternallyReparentedResponse) GetKeyspace() string { @@ -12558,7 +12831,7 @@ type UpdateCellInfoRequest struct { func (x *UpdateCellInfoRequest) Reset() { *x = UpdateCellInfoRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[200] + mi := &file_vtctldata_proto_msgTypes[205] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12571,7 +12844,7 @@ func (x *UpdateCellInfoRequest) String() string { func (*UpdateCellInfoRequest) ProtoMessage() {} func (x *UpdateCellInfoRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[200] + mi := &file_vtctldata_proto_msgTypes[205] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12584,7 +12857,7 @@ func (x *UpdateCellInfoRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellInfoRequest.ProtoReflect.Descriptor instead. func (*UpdateCellInfoRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{200} + return file_vtctldata_proto_rawDescGZIP(), []int{205} } func (x *UpdateCellInfoRequest) GetName() string { @@ -12613,7 +12886,7 @@ type UpdateCellInfoResponse struct { func (x *UpdateCellInfoResponse) Reset() { *x = UpdateCellInfoResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[201] + mi := &file_vtctldata_proto_msgTypes[206] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12626,7 +12899,7 @@ func (x *UpdateCellInfoResponse) String() string { func (*UpdateCellInfoResponse) ProtoMessage() {} func (x *UpdateCellInfoResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[201] + mi := &file_vtctldata_proto_msgTypes[206] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12639,7 +12912,7 @@ func (x *UpdateCellInfoResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellInfoResponse.ProtoReflect.Descriptor instead. func (*UpdateCellInfoResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{201} + return file_vtctldata_proto_rawDescGZIP(), []int{206} } func (x *UpdateCellInfoResponse) GetName() string { @@ -12668,7 +12941,7 @@ type UpdateCellsAliasRequest struct { func (x *UpdateCellsAliasRequest) Reset() { *x = UpdateCellsAliasRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[202] + mi := &file_vtctldata_proto_msgTypes[207] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12681,7 +12954,7 @@ func (x *UpdateCellsAliasRequest) String() string { func (*UpdateCellsAliasRequest) ProtoMessage() {} func (x *UpdateCellsAliasRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[202] + mi := &file_vtctldata_proto_msgTypes[207] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12694,7 +12967,7 @@ func (x *UpdateCellsAliasRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellsAliasRequest.ProtoReflect.Descriptor instead. func (*UpdateCellsAliasRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{202} + return file_vtctldata_proto_rawDescGZIP(), []int{207} } func (x *UpdateCellsAliasRequest) GetName() string { @@ -12723,7 +12996,7 @@ type UpdateCellsAliasResponse struct { func (x *UpdateCellsAliasResponse) Reset() { *x = UpdateCellsAliasResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[203] + mi := &file_vtctldata_proto_msgTypes[208] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12736,7 +13009,7 @@ func (x *UpdateCellsAliasResponse) String() string { func (*UpdateCellsAliasResponse) ProtoMessage() {} func (x *UpdateCellsAliasResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[203] + mi := &file_vtctldata_proto_msgTypes[208] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12749,7 +13022,7 @@ func (x *UpdateCellsAliasResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateCellsAliasResponse.ProtoReflect.Descriptor instead. func (*UpdateCellsAliasResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{203} + return file_vtctldata_proto_rawDescGZIP(), []int{208} } func (x *UpdateCellsAliasResponse) GetName() string { @@ -12777,7 +13050,7 @@ type ValidateRequest struct { func (x *ValidateRequest) Reset() { *x = ValidateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[204] + mi := &file_vtctldata_proto_msgTypes[209] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12790,7 +13063,7 @@ func (x *ValidateRequest) String() string { func (*ValidateRequest) ProtoMessage() {} func (x *ValidateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[204] + mi := &file_vtctldata_proto_msgTypes[209] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12803,7 +13076,7 @@ func (x *ValidateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateRequest.ProtoReflect.Descriptor instead. func (*ValidateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{204} + return file_vtctldata_proto_rawDescGZIP(), []int{209} } func (x *ValidateRequest) GetPingTablets() bool { @@ -12825,7 +13098,7 @@ type ValidateResponse struct { func (x *ValidateResponse) Reset() { *x = ValidateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[205] + mi := &file_vtctldata_proto_msgTypes[210] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12838,7 +13111,7 @@ func (x *ValidateResponse) String() string { func (*ValidateResponse) ProtoMessage() {} func (x *ValidateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[205] + mi := &file_vtctldata_proto_msgTypes[210] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12851,7 +13124,7 @@ func (x *ValidateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateResponse.ProtoReflect.Descriptor instead. func (*ValidateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{205} + return file_vtctldata_proto_rawDescGZIP(), []int{210} } func (x *ValidateResponse) GetResults() []string { @@ -12880,7 +13153,7 @@ type ValidateKeyspaceRequest struct { func (x *ValidateKeyspaceRequest) Reset() { *x = ValidateKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[206] + mi := &file_vtctldata_proto_msgTypes[211] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12893,7 +13166,7 @@ func (x *ValidateKeyspaceRequest) String() string { func (*ValidateKeyspaceRequest) ProtoMessage() {} func (x *ValidateKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[206] + mi := &file_vtctldata_proto_msgTypes[211] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12906,7 +13179,7 @@ func (x *ValidateKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ValidateKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{206} + return file_vtctldata_proto_rawDescGZIP(), []int{211} } func (x *ValidateKeyspaceRequest) GetKeyspace() string { @@ -12935,7 +13208,7 @@ type ValidateKeyspaceResponse struct { func (x *ValidateKeyspaceResponse) Reset() { *x = ValidateKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[207] + mi := &file_vtctldata_proto_msgTypes[212] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -12948,7 +13221,7 @@ func (x *ValidateKeyspaceResponse) String() string { func (*ValidateKeyspaceResponse) ProtoMessage() {} func (x *ValidateKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[207] + mi := &file_vtctldata_proto_msgTypes[212] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -12961,7 +13234,7 @@ func (x *ValidateKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ValidateKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{207} + return file_vtctldata_proto_rawDescGZIP(), []int{212} } func (x *ValidateKeyspaceResponse) GetResults() []string { @@ -12993,7 +13266,7 @@ type ValidateSchemaKeyspaceRequest struct { func (x *ValidateSchemaKeyspaceRequest) Reset() { *x = ValidateSchemaKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[208] + mi := &file_vtctldata_proto_msgTypes[213] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13006,7 +13279,7 @@ func (x *ValidateSchemaKeyspaceRequest) String() string { func (*ValidateSchemaKeyspaceRequest) ProtoMessage() {} func (x *ValidateSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[208] + mi := &file_vtctldata_proto_msgTypes[213] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13019,7 +13292,7 @@ func (x *ValidateSchemaKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateSchemaKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ValidateSchemaKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{208} + return file_vtctldata_proto_rawDescGZIP(), []int{213} } func (x *ValidateSchemaKeyspaceRequest) GetKeyspace() string { @@ -13069,7 +13342,7 @@ type ValidateSchemaKeyspaceResponse struct { func (x *ValidateSchemaKeyspaceResponse) Reset() { *x = ValidateSchemaKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[209] + mi := &file_vtctldata_proto_msgTypes[214] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13082,7 +13355,7 @@ func (x *ValidateSchemaKeyspaceResponse) String() string { func (*ValidateSchemaKeyspaceResponse) ProtoMessage() {} func (x *ValidateSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[209] + mi := &file_vtctldata_proto_msgTypes[214] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13095,7 +13368,7 @@ func (x *ValidateSchemaKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateSchemaKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ValidateSchemaKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{209} + return file_vtctldata_proto_rawDescGZIP(), []int{214} } func (x *ValidateSchemaKeyspaceResponse) GetResults() []string { @@ -13125,7 +13398,7 @@ type ValidateShardRequest struct { func (x *ValidateShardRequest) Reset() { *x = ValidateShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[210] + mi := &file_vtctldata_proto_msgTypes[215] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13138,7 +13411,7 @@ func (x *ValidateShardRequest) String() string { func (*ValidateShardRequest) ProtoMessage() {} func (x *ValidateShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[210] + mi := &file_vtctldata_proto_msgTypes[215] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13151,7 +13424,7 @@ func (x *ValidateShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateShardRequest.ProtoReflect.Descriptor instead. func (*ValidateShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{210} + return file_vtctldata_proto_rawDescGZIP(), []int{215} } func (x *ValidateShardRequest) GetKeyspace() string { @@ -13186,7 +13459,7 @@ type ValidateShardResponse struct { func (x *ValidateShardResponse) Reset() { *x = ValidateShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[211] + mi := &file_vtctldata_proto_msgTypes[216] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13199,7 +13472,7 @@ func (x *ValidateShardResponse) String() string { func (*ValidateShardResponse) ProtoMessage() {} func (x *ValidateShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[211] + mi := &file_vtctldata_proto_msgTypes[216] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13212,7 +13485,7 @@ func (x *ValidateShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateShardResponse.ProtoReflect.Descriptor instead. func (*ValidateShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{211} + return file_vtctldata_proto_rawDescGZIP(), []int{216} } func (x *ValidateShardResponse) GetResults() []string { @@ -13233,7 +13506,7 @@ type ValidateVersionKeyspaceRequest struct { func (x *ValidateVersionKeyspaceRequest) Reset() { *x = ValidateVersionKeyspaceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[212] + mi := &file_vtctldata_proto_msgTypes[217] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13246,7 +13519,7 @@ func (x *ValidateVersionKeyspaceRequest) String() string { func (*ValidateVersionKeyspaceRequest) ProtoMessage() {} func (x *ValidateVersionKeyspaceRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[212] + mi := &file_vtctldata_proto_msgTypes[217] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13259,7 +13532,7 @@ func (x *ValidateVersionKeyspaceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionKeyspaceRequest.ProtoReflect.Descriptor instead. func (*ValidateVersionKeyspaceRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{212} + return file_vtctldata_proto_rawDescGZIP(), []int{217} } func (x *ValidateVersionKeyspaceRequest) GetKeyspace() string { @@ -13281,7 +13554,7 @@ type ValidateVersionKeyspaceResponse struct { func (x *ValidateVersionKeyspaceResponse) Reset() { *x = ValidateVersionKeyspaceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[213] + mi := &file_vtctldata_proto_msgTypes[218] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13294,7 +13567,7 @@ func (x *ValidateVersionKeyspaceResponse) String() string { func (*ValidateVersionKeyspaceResponse) ProtoMessage() {} func (x *ValidateVersionKeyspaceResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[213] + mi := &file_vtctldata_proto_msgTypes[218] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13307,7 +13580,7 @@ func (x *ValidateVersionKeyspaceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionKeyspaceResponse.ProtoReflect.Descriptor instead. func (*ValidateVersionKeyspaceResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{213} + return file_vtctldata_proto_rawDescGZIP(), []int{218} } func (x *ValidateVersionKeyspaceResponse) GetResults() []string { @@ -13336,7 +13609,7 @@ type ValidateVersionShardRequest struct { func (x *ValidateVersionShardRequest) Reset() { *x = ValidateVersionShardRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[214] + mi := &file_vtctldata_proto_msgTypes[219] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13349,7 +13622,7 @@ func (x *ValidateVersionShardRequest) String() string { func (*ValidateVersionShardRequest) ProtoMessage() {} func (x *ValidateVersionShardRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[214] + mi := &file_vtctldata_proto_msgTypes[219] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13362,7 +13635,7 @@ func (x *ValidateVersionShardRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionShardRequest.ProtoReflect.Descriptor instead. func (*ValidateVersionShardRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{214} + return file_vtctldata_proto_rawDescGZIP(), []int{219} } func (x *ValidateVersionShardRequest) GetKeyspace() string { @@ -13390,7 +13663,7 @@ type ValidateVersionShardResponse struct { func (x *ValidateVersionShardResponse) Reset() { *x = ValidateVersionShardResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[215] + mi := &file_vtctldata_proto_msgTypes[220] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13403,7 +13676,7 @@ func (x *ValidateVersionShardResponse) String() string { func (*ValidateVersionShardResponse) ProtoMessage() {} func (x *ValidateVersionShardResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[215] + mi := &file_vtctldata_proto_msgTypes[220] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13416,7 +13689,7 @@ func (x *ValidateVersionShardResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVersionShardResponse.ProtoReflect.Descriptor instead. func (*ValidateVersionShardResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{215} + return file_vtctldata_proto_rawDescGZIP(), []int{220} } func (x *ValidateVersionShardResponse) GetResults() []string { @@ -13440,7 +13713,7 @@ type ValidateVSchemaRequest struct { func (x *ValidateVSchemaRequest) Reset() { *x = ValidateVSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[216] + mi := &file_vtctldata_proto_msgTypes[221] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13453,7 +13726,7 @@ func (x *ValidateVSchemaRequest) String() string { func (*ValidateVSchemaRequest) ProtoMessage() {} func (x *ValidateVSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[216] + mi := &file_vtctldata_proto_msgTypes[221] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13466,7 +13739,7 @@ func (x *ValidateVSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVSchemaRequest.ProtoReflect.Descriptor instead. func (*ValidateVSchemaRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{216} + return file_vtctldata_proto_rawDescGZIP(), []int{221} } func (x *ValidateVSchemaRequest) GetKeyspace() string { @@ -13509,7 +13782,7 @@ type ValidateVSchemaResponse struct { func (x *ValidateVSchemaResponse) Reset() { *x = ValidateVSchemaResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[217] + mi := &file_vtctldata_proto_msgTypes[222] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13522,7 +13795,7 @@ func (x *ValidateVSchemaResponse) String() string { func (*ValidateVSchemaResponse) ProtoMessage() {} func (x *ValidateVSchemaResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[217] + mi := &file_vtctldata_proto_msgTypes[222] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13535,7 +13808,7 @@ func (x *ValidateVSchemaResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ValidateVSchemaResponse.ProtoReflect.Descriptor instead. func (*ValidateVSchemaResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{217} + return file_vtctldata_proto_rawDescGZIP(), []int{222} } func (x *ValidateVSchemaResponse) GetResults() []string { @@ -13582,7 +13855,7 @@ type VDiffCreateRequest struct { func (x *VDiffCreateRequest) Reset() { *x = VDiffCreateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[218] + mi := &file_vtctldata_proto_msgTypes[223] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13595,7 +13868,7 @@ func (x *VDiffCreateRequest) String() string { func (*VDiffCreateRequest) ProtoMessage() {} func (x *VDiffCreateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[218] + mi := &file_vtctldata_proto_msgTypes[223] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13608,7 +13881,7 @@ func (x *VDiffCreateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffCreateRequest.ProtoReflect.Descriptor instead. func (*VDiffCreateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{218} + return file_vtctldata_proto_rawDescGZIP(), []int{223} } func (x *VDiffCreateRequest) GetWorkflow() string { @@ -13764,7 +14037,7 @@ type VDiffCreateResponse struct { func (x *VDiffCreateResponse) Reset() { *x = VDiffCreateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[219] + mi := &file_vtctldata_proto_msgTypes[224] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13777,7 +14050,7 @@ func (x *VDiffCreateResponse) String() string { func (*VDiffCreateResponse) ProtoMessage() {} func (x *VDiffCreateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[219] + mi := &file_vtctldata_proto_msgTypes[224] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13790,7 +14063,7 @@ func (x *VDiffCreateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffCreateResponse.ProtoReflect.Descriptor instead. func (*VDiffCreateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{219} + return file_vtctldata_proto_rawDescGZIP(), []int{224} } func (x *VDiffCreateResponse) GetUUID() string { @@ -13814,7 +14087,7 @@ type VDiffDeleteRequest struct { func (x *VDiffDeleteRequest) Reset() { *x = VDiffDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[220] + mi := &file_vtctldata_proto_msgTypes[225] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13827,7 +14100,7 @@ func (x *VDiffDeleteRequest) String() string { func (*VDiffDeleteRequest) ProtoMessage() {} func (x *VDiffDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[220] + mi := &file_vtctldata_proto_msgTypes[225] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13840,7 +14113,7 @@ func (x *VDiffDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffDeleteRequest.ProtoReflect.Descriptor instead. func (*VDiffDeleteRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{220} + return file_vtctldata_proto_rawDescGZIP(), []int{225} } func (x *VDiffDeleteRequest) GetWorkflow() string { @@ -13873,7 +14146,7 @@ type VDiffDeleteResponse struct { func (x *VDiffDeleteResponse) Reset() { *x = VDiffDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[221] + mi := &file_vtctldata_proto_msgTypes[226] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13886,7 +14159,7 @@ func (x *VDiffDeleteResponse) String() string { func (*VDiffDeleteResponse) ProtoMessage() {} func (x *VDiffDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[221] + mi := &file_vtctldata_proto_msgTypes[226] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13899,7 +14172,7 @@ func (x *VDiffDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffDeleteResponse.ProtoReflect.Descriptor instead. func (*VDiffDeleteResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{221} + return file_vtctldata_proto_rawDescGZIP(), []int{226} } type VDiffResumeRequest struct { @@ -13915,7 +14188,7 @@ type VDiffResumeRequest struct { func (x *VDiffResumeRequest) Reset() { *x = VDiffResumeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[222] + mi := &file_vtctldata_proto_msgTypes[227] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13928,7 +14201,7 @@ func (x *VDiffResumeRequest) String() string { func (*VDiffResumeRequest) ProtoMessage() {} func (x *VDiffResumeRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[222] + mi := &file_vtctldata_proto_msgTypes[227] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -13941,7 +14214,7 @@ func (x *VDiffResumeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffResumeRequest.ProtoReflect.Descriptor instead. func (*VDiffResumeRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{222} + return file_vtctldata_proto_rawDescGZIP(), []int{227} } func (x *VDiffResumeRequest) GetWorkflow() string { @@ -13974,7 +14247,7 @@ type VDiffResumeResponse struct { func (x *VDiffResumeResponse) Reset() { *x = VDiffResumeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[223] + mi := &file_vtctldata_proto_msgTypes[228] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -13987,7 +14260,7 @@ func (x *VDiffResumeResponse) String() string { func (*VDiffResumeResponse) ProtoMessage() {} func (x *VDiffResumeResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[223] + mi := &file_vtctldata_proto_msgTypes[228] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14000,7 +14273,7 @@ func (x *VDiffResumeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffResumeResponse.ProtoReflect.Descriptor instead. func (*VDiffResumeResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{223} + return file_vtctldata_proto_rawDescGZIP(), []int{228} } type VDiffShowRequest struct { @@ -14017,7 +14290,7 @@ type VDiffShowRequest struct { func (x *VDiffShowRequest) Reset() { *x = VDiffShowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[224] + mi := &file_vtctldata_proto_msgTypes[229] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14030,7 +14303,7 @@ func (x *VDiffShowRequest) String() string { func (*VDiffShowRequest) ProtoMessage() {} func (x *VDiffShowRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[224] + mi := &file_vtctldata_proto_msgTypes[229] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14043,7 +14316,7 @@ func (x *VDiffShowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffShowRequest.ProtoReflect.Descriptor instead. func (*VDiffShowRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{224} + return file_vtctldata_proto_rawDescGZIP(), []int{229} } func (x *VDiffShowRequest) GetWorkflow() string { @@ -14079,7 +14352,7 @@ type VDiffShowResponse struct { func (x *VDiffShowResponse) Reset() { *x = VDiffShowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[225] + mi := &file_vtctldata_proto_msgTypes[230] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14092,7 +14365,7 @@ func (x *VDiffShowResponse) String() string { func (*VDiffShowResponse) ProtoMessage() {} func (x *VDiffShowResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[225] + mi := &file_vtctldata_proto_msgTypes[230] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14105,7 +14378,7 @@ func (x *VDiffShowResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffShowResponse.ProtoReflect.Descriptor instead. func (*VDiffShowResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{225} + return file_vtctldata_proto_rawDescGZIP(), []int{230} } func (x *VDiffShowResponse) GetTabletResponses() map[string]*tabletmanagerdata.VDiffResponse { @@ -14128,7 +14401,7 @@ type VDiffStopRequest struct { func (x *VDiffStopRequest) Reset() { *x = VDiffStopRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[226] + mi := &file_vtctldata_proto_msgTypes[231] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14141,7 +14414,7 @@ func (x *VDiffStopRequest) String() string { func (*VDiffStopRequest) ProtoMessage() {} func (x *VDiffStopRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[226] + mi := &file_vtctldata_proto_msgTypes[231] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14154,7 +14427,7 @@ func (x *VDiffStopRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffStopRequest.ProtoReflect.Descriptor instead. func (*VDiffStopRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{226} + return file_vtctldata_proto_rawDescGZIP(), []int{231} } func (x *VDiffStopRequest) GetWorkflow() string { @@ -14187,7 +14460,7 @@ type VDiffStopResponse struct { func (x *VDiffStopResponse) Reset() { *x = VDiffStopResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[227] + mi := &file_vtctldata_proto_msgTypes[232] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14200,7 +14473,7 @@ func (x *VDiffStopResponse) String() string { func (*VDiffStopResponse) ProtoMessage() {} func (x *VDiffStopResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[227] + mi := &file_vtctldata_proto_msgTypes[232] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14213,7 +14486,7 @@ func (x *VDiffStopResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffStopResponse.ProtoReflect.Descriptor instead. func (*VDiffStopResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{227} + return file_vtctldata_proto_rawDescGZIP(), []int{232} } type WorkflowDeleteRequest struct { @@ -14231,7 +14504,7 @@ type WorkflowDeleteRequest struct { func (x *WorkflowDeleteRequest) Reset() { *x = WorkflowDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[228] + mi := &file_vtctldata_proto_msgTypes[233] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14244,7 +14517,7 @@ func (x *WorkflowDeleteRequest) String() string { func (*WorkflowDeleteRequest) ProtoMessage() {} func (x *WorkflowDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[228] + mi := &file_vtctldata_proto_msgTypes[233] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14257,7 +14530,7 @@ func (x *WorkflowDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowDeleteRequest.ProtoReflect.Descriptor instead. func (*WorkflowDeleteRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{228} + return file_vtctldata_proto_rawDescGZIP(), []int{233} } func (x *WorkflowDeleteRequest) GetKeyspace() string { @@ -14307,7 +14580,7 @@ type WorkflowDeleteResponse struct { func (x *WorkflowDeleteResponse) Reset() { *x = WorkflowDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[229] + mi := &file_vtctldata_proto_msgTypes[234] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14320,7 +14593,7 @@ func (x *WorkflowDeleteResponse) String() string { func (*WorkflowDeleteResponse) ProtoMessage() {} func (x *WorkflowDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[229] + mi := &file_vtctldata_proto_msgTypes[234] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14333,7 +14606,7 @@ func (x *WorkflowDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowDeleteResponse.ProtoReflect.Descriptor instead. func (*WorkflowDeleteResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{229} + return file_vtctldata_proto_rawDescGZIP(), []int{234} } func (x *WorkflowDeleteResponse) GetSummary() string { @@ -14363,7 +14636,7 @@ type WorkflowStatusRequest struct { func (x *WorkflowStatusRequest) Reset() { *x = WorkflowStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[230] + mi := &file_vtctldata_proto_msgTypes[235] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14376,7 +14649,7 @@ func (x *WorkflowStatusRequest) String() string { func (*WorkflowStatusRequest) ProtoMessage() {} func (x *WorkflowStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[230] + mi := &file_vtctldata_proto_msgTypes[235] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14389,7 +14662,7 @@ func (x *WorkflowStatusRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowStatusRequest.ProtoReflect.Descriptor instead. func (*WorkflowStatusRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{230} + return file_vtctldata_proto_rawDescGZIP(), []int{235} } func (x *WorkflowStatusRequest) GetKeyspace() string { @@ -14427,7 +14700,7 @@ type WorkflowStatusResponse struct { func (x *WorkflowStatusResponse) Reset() { *x = WorkflowStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[231] + mi := &file_vtctldata_proto_msgTypes[236] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14440,7 +14713,7 @@ func (x *WorkflowStatusResponse) String() string { func (*WorkflowStatusResponse) ProtoMessage() {} func (x *WorkflowStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[231] + mi := &file_vtctldata_proto_msgTypes[236] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14453,7 +14726,7 @@ func (x *WorkflowStatusResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowStatusResponse.ProtoReflect.Descriptor instead. func (*WorkflowStatusResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{231} + return file_vtctldata_proto_rawDescGZIP(), []int{236} } func (x *WorkflowStatusResponse) GetTableCopyState() map[string]*WorkflowStatusResponse_TableCopyState { @@ -14498,7 +14771,7 @@ type WorkflowSwitchTrafficRequest struct { func (x *WorkflowSwitchTrafficRequest) Reset() { *x = WorkflowSwitchTrafficRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[232] + mi := &file_vtctldata_proto_msgTypes[237] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14511,7 +14784,7 @@ func (x *WorkflowSwitchTrafficRequest) String() string { func (*WorkflowSwitchTrafficRequest) ProtoMessage() {} func (x *WorkflowSwitchTrafficRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[232] + mi := &file_vtctldata_proto_msgTypes[237] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14524,7 +14797,7 @@ func (x *WorkflowSwitchTrafficRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowSwitchTrafficRequest.ProtoReflect.Descriptor instead. func (*WorkflowSwitchTrafficRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{232} + return file_vtctldata_proto_rawDescGZIP(), []int{237} } func (x *WorkflowSwitchTrafficRequest) GetKeyspace() string { @@ -14618,7 +14891,7 @@ type WorkflowSwitchTrafficResponse struct { func (x *WorkflowSwitchTrafficResponse) Reset() { *x = WorkflowSwitchTrafficResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[233] + mi := &file_vtctldata_proto_msgTypes[238] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14631,7 +14904,7 @@ func (x *WorkflowSwitchTrafficResponse) String() string { func (*WorkflowSwitchTrafficResponse) ProtoMessage() {} func (x *WorkflowSwitchTrafficResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[233] + mi := &file_vtctldata_proto_msgTypes[238] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14644,7 +14917,7 @@ func (x *WorkflowSwitchTrafficResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowSwitchTrafficResponse.ProtoReflect.Descriptor instead. func (*WorkflowSwitchTrafficResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{233} + return file_vtctldata_proto_rawDescGZIP(), []int{238} } func (x *WorkflowSwitchTrafficResponse) GetSummary() string { @@ -14689,7 +14962,7 @@ type WorkflowUpdateRequest struct { func (x *WorkflowUpdateRequest) Reset() { *x = WorkflowUpdateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[234] + mi := &file_vtctldata_proto_msgTypes[239] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14702,7 +14975,7 @@ func (x *WorkflowUpdateRequest) String() string { func (*WorkflowUpdateRequest) ProtoMessage() {} func (x *WorkflowUpdateRequest) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[234] + mi := &file_vtctldata_proto_msgTypes[239] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14715,7 +14988,7 @@ func (x *WorkflowUpdateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowUpdateRequest.ProtoReflect.Descriptor instead. func (*WorkflowUpdateRequest) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{234} + return file_vtctldata_proto_rawDescGZIP(), []int{239} } func (x *WorkflowUpdateRequest) GetKeyspace() string { @@ -14744,7 +15017,7 @@ type WorkflowUpdateResponse struct { func (x *WorkflowUpdateResponse) Reset() { *x = WorkflowUpdateResponse{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[235] + mi := &file_vtctldata_proto_msgTypes[240] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14757,7 +15030,7 @@ func (x *WorkflowUpdateResponse) String() string { func (*WorkflowUpdateResponse) ProtoMessage() {} func (x *WorkflowUpdateResponse) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[235] + mi := &file_vtctldata_proto_msgTypes[240] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14770,7 +15043,7 @@ func (x *WorkflowUpdateResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowUpdateResponse.ProtoReflect.Descriptor instead. func (*WorkflowUpdateResponse) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{235} + return file_vtctldata_proto_rawDescGZIP(), []int{240} } func (x *WorkflowUpdateResponse) GetSummary() string { @@ -14799,7 +15072,7 @@ type Workflow_ReplicationLocation struct { func (x *Workflow_ReplicationLocation) Reset() { *x = Workflow_ReplicationLocation{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[237] + mi := &file_vtctldata_proto_msgTypes[242] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14812,7 +15085,7 @@ func (x *Workflow_ReplicationLocation) String() string { func (*Workflow_ReplicationLocation) ProtoMessage() {} func (x *Workflow_ReplicationLocation) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[237] + mi := &file_vtctldata_proto_msgTypes[242] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14825,7 +15098,7 @@ func (x *Workflow_ReplicationLocation) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow_ReplicationLocation.ProtoReflect.Descriptor instead. func (*Workflow_ReplicationLocation) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7, 1} + return file_vtctldata_proto_rawDescGZIP(), []int{8, 1} } func (x *Workflow_ReplicationLocation) GetKeyspace() string { @@ -14855,7 +15128,7 @@ type Workflow_ShardStream struct { func (x *Workflow_ShardStream) Reset() { *x = Workflow_ShardStream{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[238] + mi := &file_vtctldata_proto_msgTypes[243] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14868,7 +15141,7 @@ func (x *Workflow_ShardStream) String() string { func (*Workflow_ShardStream) ProtoMessage() {} func (x *Workflow_ShardStream) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[238] + mi := &file_vtctldata_proto_msgTypes[243] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14881,7 +15154,7 @@ func (x *Workflow_ShardStream) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow_ShardStream.ProtoReflect.Descriptor instead. func (*Workflow_ShardStream) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7, 2} + return file_vtctldata_proto_rawDescGZIP(), []int{8, 2} } func (x *Workflow_ShardStream) GetStreams() []*Workflow_Stream { @@ -14943,7 +15216,7 @@ type Workflow_Stream struct { func (x *Workflow_Stream) Reset() { *x = Workflow_Stream{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[239] + mi := &file_vtctldata_proto_msgTypes[244] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -14956,7 +15229,7 @@ func (x *Workflow_Stream) String() string { func (*Workflow_Stream) ProtoMessage() {} func (x *Workflow_Stream) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[239] + mi := &file_vtctldata_proto_msgTypes[244] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -14969,7 +15242,7 @@ func (x *Workflow_Stream) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow_Stream.ProtoReflect.Descriptor instead. func (*Workflow_Stream) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7, 3} + return file_vtctldata_proto_rawDescGZIP(), []int{8, 3} } func (x *Workflow_Stream) GetId() int64 { @@ -15125,7 +15398,7 @@ type Workflow_Stream_CopyState struct { func (x *Workflow_Stream_CopyState) Reset() { *x = Workflow_Stream_CopyState{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[240] + mi := &file_vtctldata_proto_msgTypes[245] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15138,7 +15411,7 @@ func (x *Workflow_Stream_CopyState) String() string { func (*Workflow_Stream_CopyState) ProtoMessage() {} func (x *Workflow_Stream_CopyState) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[240] + mi := &file_vtctldata_proto_msgTypes[245] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15151,7 +15424,7 @@ func (x *Workflow_Stream_CopyState) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow_Stream_CopyState.ProtoReflect.Descriptor instead. func (*Workflow_Stream_CopyState) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7, 3, 0} + return file_vtctldata_proto_rawDescGZIP(), []int{8, 3, 0} } func (x *Workflow_Stream_CopyState) GetTable() string { @@ -15193,7 +15466,7 @@ type Workflow_Stream_Log struct { func (x *Workflow_Stream_Log) Reset() { *x = Workflow_Stream_Log{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[241] + mi := &file_vtctldata_proto_msgTypes[246] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15206,7 +15479,7 @@ func (x *Workflow_Stream_Log) String() string { func (*Workflow_Stream_Log) ProtoMessage() {} func (x *Workflow_Stream_Log) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[241] + mi := &file_vtctldata_proto_msgTypes[246] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15219,7 +15492,7 @@ func (x *Workflow_Stream_Log) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow_Stream_Log.ProtoReflect.Descriptor instead. func (*Workflow_Stream_Log) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7, 3, 1} + return file_vtctldata_proto_rawDescGZIP(), []int{8, 3, 1} } func (x *Workflow_Stream_Log) GetId() int64 { @@ -15290,7 +15563,7 @@ type Workflow_Stream_ThrottlerStatus struct { func (x *Workflow_Stream_ThrottlerStatus) Reset() { *x = Workflow_Stream_ThrottlerStatus{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[242] + mi := &file_vtctldata_proto_msgTypes[247] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15303,7 +15576,7 @@ func (x *Workflow_Stream_ThrottlerStatus) String() string { func (*Workflow_Stream_ThrottlerStatus) ProtoMessage() {} func (x *Workflow_Stream_ThrottlerStatus) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[242] + mi := &file_vtctldata_proto_msgTypes[247] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15316,7 +15589,7 @@ func (x *Workflow_Stream_ThrottlerStatus) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow_Stream_ThrottlerStatus.ProtoReflect.Descriptor instead. func (*Workflow_Stream_ThrottlerStatus) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{7, 3, 2} + return file_vtctldata_proto_rawDescGZIP(), []int{8, 3, 2} } func (x *Workflow_Stream_ThrottlerStatus) GetComponentThrottled() string { @@ -15344,7 +15617,7 @@ type ApplyVSchemaResponse_ParamList struct { func (x *ApplyVSchemaResponse_ParamList) Reset() { *x = ApplyVSchemaResponse_ParamList{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[245] + mi := &file_vtctldata_proto_msgTypes[250] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15357,7 +15630,7 @@ func (x *ApplyVSchemaResponse_ParamList) String() string { func (*ApplyVSchemaResponse_ParamList) ProtoMessage() {} func (x *ApplyVSchemaResponse_ParamList) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[245] + mi := &file_vtctldata_proto_msgTypes[250] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15370,7 +15643,7 @@ func (x *ApplyVSchemaResponse_ParamList) ProtoReflect() protoreflect.Message { // Deprecated: Use ApplyVSchemaResponse_ParamList.ProtoReflect.Descriptor instead. func (*ApplyVSchemaResponse_ParamList) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{19, 1} + return file_vtctldata_proto_rawDescGZIP(), []int{22, 1} } func (x *ApplyVSchemaResponse_ParamList) GetParams() []string { @@ -15391,7 +15664,7 @@ type GetSrvKeyspaceNamesResponse_NameList struct { func (x *GetSrvKeyspaceNamesResponse_NameList) Reset() { *x = GetSrvKeyspaceNamesResponse_NameList{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[254] + mi := &file_vtctldata_proto_msgTypes[259] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15404,7 +15677,7 @@ func (x *GetSrvKeyspaceNamesResponse_NameList) String() string { func (*GetSrvKeyspaceNamesResponse_NameList) ProtoMessage() {} func (x *GetSrvKeyspaceNamesResponse_NameList) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[254] + mi := &file_vtctldata_proto_msgTypes[259] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15417,7 +15690,7 @@ func (x *GetSrvKeyspaceNamesResponse_NameList) ProtoReflect() protoreflect.Messa // Deprecated: Use GetSrvKeyspaceNamesResponse_NameList.ProtoReflect.Descriptor instead. func (*GetSrvKeyspaceNamesResponse_NameList) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{90, 1} + return file_vtctldata_proto_rawDescGZIP(), []int{95, 1} } func (x *GetSrvKeyspaceNamesResponse_NameList) GetNames() []string { @@ -15440,7 +15713,7 @@ type MoveTablesCreateResponse_TabletInfo struct { func (x *MoveTablesCreateResponse_TabletInfo) Reset() { *x = MoveTablesCreateResponse_TabletInfo{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[258] + mi := &file_vtctldata_proto_msgTypes[263] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15453,7 +15726,7 @@ func (x *MoveTablesCreateResponse_TabletInfo) String() string { func (*MoveTablesCreateResponse_TabletInfo) ProtoMessage() {} func (x *MoveTablesCreateResponse_TabletInfo) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[258] + mi := &file_vtctldata_proto_msgTypes[263] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15466,7 +15739,7 @@ func (x *MoveTablesCreateResponse_TabletInfo) ProtoReflect() protoreflect.Messag // Deprecated: Use MoveTablesCreateResponse_TabletInfo.ProtoReflect.Descriptor instead. func (*MoveTablesCreateResponse_TabletInfo) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{134, 0} + return file_vtctldata_proto_rawDescGZIP(), []int{139, 0} } func (x *MoveTablesCreateResponse_TabletInfo) GetTablet() *topodata.TabletAlias { @@ -15496,7 +15769,7 @@ type WorkflowDeleteResponse_TabletInfo struct { func (x *WorkflowDeleteResponse_TabletInfo) Reset() { *x = WorkflowDeleteResponse_TabletInfo{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[268] + mi := &file_vtctldata_proto_msgTypes[273] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15509,7 +15782,7 @@ func (x *WorkflowDeleteResponse_TabletInfo) String() string { func (*WorkflowDeleteResponse_TabletInfo) ProtoMessage() {} func (x *WorkflowDeleteResponse_TabletInfo) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[268] + mi := &file_vtctldata_proto_msgTypes[273] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15522,7 +15795,7 @@ func (x *WorkflowDeleteResponse_TabletInfo) ProtoReflect() protoreflect.Message // Deprecated: Use WorkflowDeleteResponse_TabletInfo.ProtoReflect.Descriptor instead. func (*WorkflowDeleteResponse_TabletInfo) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{229, 0} + return file_vtctldata_proto_rawDescGZIP(), []int{234, 0} } func (x *WorkflowDeleteResponse_TabletInfo) GetTablet() *topodata.TabletAlias { @@ -15555,7 +15828,7 @@ type WorkflowStatusResponse_TableCopyState struct { func (x *WorkflowStatusResponse_TableCopyState) Reset() { *x = WorkflowStatusResponse_TableCopyState{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[269] + mi := &file_vtctldata_proto_msgTypes[274] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15568,7 +15841,7 @@ func (x *WorkflowStatusResponse_TableCopyState) String() string { func (*WorkflowStatusResponse_TableCopyState) ProtoMessage() {} func (x *WorkflowStatusResponse_TableCopyState) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[269] + mi := &file_vtctldata_proto_msgTypes[274] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15581,7 +15854,7 @@ func (x *WorkflowStatusResponse_TableCopyState) ProtoReflect() protoreflect.Mess // Deprecated: Use WorkflowStatusResponse_TableCopyState.ProtoReflect.Descriptor instead. func (*WorkflowStatusResponse_TableCopyState) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{231, 0} + return file_vtctldata_proto_rawDescGZIP(), []int{236, 0} } func (x *WorkflowStatusResponse_TableCopyState) GetRowsCopied() int64 { @@ -15642,7 +15915,7 @@ type WorkflowStatusResponse_ShardStreamState struct { func (x *WorkflowStatusResponse_ShardStreamState) Reset() { *x = WorkflowStatusResponse_ShardStreamState{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[270] + mi := &file_vtctldata_proto_msgTypes[275] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15655,7 +15928,7 @@ func (x *WorkflowStatusResponse_ShardStreamState) String() string { func (*WorkflowStatusResponse_ShardStreamState) ProtoMessage() {} func (x *WorkflowStatusResponse_ShardStreamState) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[270] + mi := &file_vtctldata_proto_msgTypes[275] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15668,7 +15941,7 @@ func (x *WorkflowStatusResponse_ShardStreamState) ProtoReflect() protoreflect.Me // Deprecated: Use WorkflowStatusResponse_ShardStreamState.ProtoReflect.Descriptor instead. func (*WorkflowStatusResponse_ShardStreamState) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{231, 1} + return file_vtctldata_proto_rawDescGZIP(), []int{236, 1} } func (x *WorkflowStatusResponse_ShardStreamState) GetId() int32 { @@ -15724,7 +15997,7 @@ type WorkflowStatusResponse_ShardStreams struct { func (x *WorkflowStatusResponse_ShardStreams) Reset() { *x = WorkflowStatusResponse_ShardStreams{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[271] + mi := &file_vtctldata_proto_msgTypes[276] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15737,7 +16010,7 @@ func (x *WorkflowStatusResponse_ShardStreams) String() string { func (*WorkflowStatusResponse_ShardStreams) ProtoMessage() {} func (x *WorkflowStatusResponse_ShardStreams) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[271] + mi := &file_vtctldata_proto_msgTypes[276] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15750,7 +16023,7 @@ func (x *WorkflowStatusResponse_ShardStreams) ProtoReflect() protoreflect.Messag // Deprecated: Use WorkflowStatusResponse_ShardStreams.ProtoReflect.Descriptor instead. func (*WorkflowStatusResponse_ShardStreams) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{231, 2} + return file_vtctldata_proto_rawDescGZIP(), []int{236, 2} } func (x *WorkflowStatusResponse_ShardStreams) GetStreams() []*WorkflowStatusResponse_ShardStreamState { @@ -15774,7 +16047,7 @@ type WorkflowUpdateResponse_TabletInfo struct { func (x *WorkflowUpdateResponse_TabletInfo) Reset() { *x = WorkflowUpdateResponse_TabletInfo{} if protoimpl.UnsafeEnabled { - mi := &file_vtctldata_proto_msgTypes[274] + mi := &file_vtctldata_proto_msgTypes[279] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -15787,7 +16060,7 @@ func (x *WorkflowUpdateResponse_TabletInfo) String() string { func (*WorkflowUpdateResponse_TabletInfo) ProtoMessage() {} func (x *WorkflowUpdateResponse_TabletInfo) ProtoReflect() protoreflect.Message { - mi := &file_vtctldata_proto_msgTypes[274] + mi := &file_vtctldata_proto_msgTypes[279] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -15800,7 +16073,7 @@ func (x *WorkflowUpdateResponse_TabletInfo) ProtoReflect() protoreflect.Message // Deprecated: Use WorkflowUpdateResponse_TabletInfo.ProtoReflect.Descriptor instead. func (*WorkflowUpdateResponse_TabletInfo) Descriptor() ([]byte, []int) { - return file_vtctldata_proto_rawDescGZIP(), []int{235, 0} + return file_vtctldata_proto_rawDescGZIP(), []int{240, 0} } func (x *WorkflowUpdateResponse_TabletInfo) GetTablet() *topodata.TabletAlias { @@ -15850,7 +16123,7 @@ 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, 0x83, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x64, 0x6c, 0x22, 0xca, 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, @@ -15899,529 +16172,538 @@ var file_vtctldata_proto_rawDesc = []byte{ 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 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, 0xc0, 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, 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, 0x12, 0x39, 0x0a, 0x19, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f, - 0x66, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x73, 0x18, 0x36, 0x20, 0x01, 0x28, 0x09, 0x52, 0x16, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x64, - 0x46, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x4b, 0x65, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x73, 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, 0x99, 0x11, 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, 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, 0x12, 0x48, 0x0a, 0x21, 0x6d, 0x61, - 0x78, 0x5f, 0x76, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1d, 0x6d, 0x61, 0x78, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x4c, 0x61, 0x67, 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, 0x09, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, 0x66, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, - 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 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, - 0xc1, 0x0a, 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, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, - 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, - 0x70, 0x69, 0x65, 0x64, 0x12, 0x55, 0x0a, 0x10, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, - 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, + 0x43, 0x6f, 0x70, 0x79, 0x12, 0x45, 0x0a, 0x10, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x11, 0x20, 0x01, 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, 0x2e, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0f, 0x74, 0x68, 0x72, 0x6f, - 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x12, 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, 0x13, 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, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x14, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x1a, 0x57, 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, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x69, - 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x49, - 0x64, 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, + 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0xc0, 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, - 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, 0x1a, 0x77, 0x0a, 0x0f, 0x54, 0x68, - 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2f, 0x0a, - 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x64, 0x18, 0x01, 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, 0x33, - 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, - 0x6c, 0x65, 0x64, 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, 0xce, 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, 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, 0x4a, 0x04, 0x08, 0x08, 0x10, 0x09, 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, 0xdb, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, - 0x63, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, - 0x22, 0xca, 0x02, 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, 0x12, 0x6c, 0x0a, 0x15, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, - 0x77, 0x6e, 0x5f, 0x76, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x56, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x13, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, - 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x71, 0x0a, 0x18, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, - 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x3f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, - 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x23, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x61, - 0x6d, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 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, 0x05, 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, 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, 0x05, 0x52, 0x0b, 0x63, - 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x75, 0x70, + 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, 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, 0x12, 0x39, 0x0a, 0x19, 0x72, + 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f, 0x66, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x5f, 0x6b, + 0x65, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x36, 0x20, 0x01, 0x28, 0x09, 0x52, 0x16, + 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x46, 0x6f, 0x72, 0x65, 0x69, 0x67, 0x6e, 0x4b, 0x65, + 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x73, 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, 0x62, + 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x32, + 0x0a, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x41, 0x6c, 0x69, + 0x61, 0x73, 0x22, 0xcf, 0x11, 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, 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, 0x12, 0x48, 0x0a, 0x21, 0x6d, 0x61, 0x78, 0x5f, 0x76, 0x5f, 0x72, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x1d, 0x6d, 0x61, 0x78, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 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, 0x09, 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, 0x34, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, + 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0xc1, 0x0a, 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, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, + 0x69, 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, + 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x55, 0x0a, 0x10, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, + 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2a, 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, 0x54, 0x68, 0x72, 0x6f, + 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x0f, 0x74, 0x68, 0x72, + 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x37, 0x0a, 0x0c, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x12, 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, 0x13, 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, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x14, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x1a, 0x57, 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, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, + 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x49, 0x64, 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, 0x1a, 0x77, 0x0a, 0x0f, 0x54, + 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2f, + 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, + 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x01, 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, + 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, + 0x74, 0x6c, 0x65, 0x64, 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, 0xbf, 0x01, 0x0a, 0x20, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x53, 0x0a, 0x16, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x5f, 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, + 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x14, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 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, 0x23, 0x0a, 0x21, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 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, 0xce, + 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, 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, 0x4a, 0x04, 0x08, 0x08, 0x10, 0x09, 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, 0xdb, 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, + 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x22, 0xca, 0x02, 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, 0x12, + 0x6c, 0x0a, 0x15, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x5f, 0x76, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, + 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, + 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, + 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, + 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x71, 0x0a, + 0x18, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3f, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x50, 0x61, 0x72, 0x61, + 0x6d, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x1a, 0x23, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, + 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x70, + 0x61, 0x72, 0x61, 0x6d, 0x73, 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, 0x05, 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, 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, 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, + 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, 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, 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, + 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, 0x05, 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, 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, + 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, 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, @@ -16430,1712 +16712,1746 @@ var file_vtctldata_proto_rawDesc = []byte{ 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, 0xdd, 0x02, 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, 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, 0x4a, - 0x04, 0x08, 0x06, 0x10, 0x07, 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, + 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, 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, + 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, 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, 0xdd, 0x02, + 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, 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, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 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, 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, 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, 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, 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, + 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, 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, 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, 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, 0xd4, 0x01, 0x0a, 0x1d, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, - 0x44, 0x42, 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x38, 0x0a, 0x0c, 0x74, 0x61, + 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, 0x10, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x6f, + 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, 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, - 0x4e, 0x0a, 0x1e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x46, - 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x2c, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 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, 0x54, - 0x0a, 0x22, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 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, 0xeb, 0x01, 0x0a, 0x23, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, - 0x74, 0x4f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7c, 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, 0x47, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, - 0x74, 0x4f, 0x76, 0x65, 0x72, 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, 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, 0x64, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 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, 0xd4, 0x01, 0x0a, 0x1d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, + 0x69, 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, 0x10, 0x0a, 0x03, + 0x73, 0x71, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x73, 0x71, 0x6c, 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, 0x4e, 0x0a, 0x1e, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, + 0x41, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x07, 0x72, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 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, 0x54, 0x0a, 0x22, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, + 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 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, 0xeb, 0x01, 0x0a, + 0x23, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7c, 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, 0x47, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 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, 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, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x22, 0x83, 0x02, 0x0a, 0x1b, 0x47, 0x65, 0x74, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7d, 0x0a, 0x19, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x62, 0x79, - 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x76, 0x74, - 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x16, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x42, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x1a, 0x65, 0x0a, 0x1b, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x43, 0x65, 0x6c, - 0x6c, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x30, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 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, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 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, + 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, 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, 0xc6, 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, + 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, 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, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6c, 0x6f, 0x67, - 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x06, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 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, 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, 0x20, + 0x0a, 0x1e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, + 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x22, 0x76, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, + 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x53, 0x0a, 0x16, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, + 0x72, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, + 0x65, 0x73, 0x52, 0x14, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, + 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 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, 0x64, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 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, 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, 0x02, 0x0a, 0x1b, 0x47, + 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7d, 0x0a, 0x19, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x62, 0x79, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x16, 0x73, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x43, 0x65, 0x6c, 0x6c, 0x1a, 0x65, 0x0a, 0x1b, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x43, + 0x65, 0x6c, 0x6c, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x30, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 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, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 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, - 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, 0xff, 0x02, 0x0a, 0x19, - 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 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, 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, 0x29, 0x0a, 0x06, 0x76, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x06, 0x76, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x12, 0x42, 0x0a, 0x1e, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x5f, 0x61, - 0x66, 0x74, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x6f, - 0x77, 0x6e, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1a, 0x63, 0x6f, 0x6e, 0x74, - 0x69, 0x6e, 0x75, 0x65, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 0x57, 0x69, 0x74, - 0x68, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x06, 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, 0x07, - 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, 0x22, 0x1c, 0x0a, - 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x77, 0x0a, 0x1e, 0x4c, - 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 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, 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, 0xc6, 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, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, - 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x22, 0x4c, 0x0a, 0x1f, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x64, 0x22, 0x56, 0x0a, 0x18, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, - 0x0a, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x74, - 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, - 0x52, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x22, 0x1b, 0x0a, 0x19, 0x4d, 0x61, - 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xdd, 0x05, 0x0a, 0x14, 0x4d, 0x69, 0x67, 0x72, - 0x61, 0x74, 0x65, 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, 0x1d, - 0x0a, 0x0a, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x05, 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, 0x06, 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, 0x07, 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, 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, 0x28, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x18, 0x0b, 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, 0x0c, 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, 0x0d, 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, 0x0e, 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, 0x0f, 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, 0x10, 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, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6e, 0x6f, 0x52, 0x6f, 0x75, 0x74, 0x69, - 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0xe6, 0x01, 0x0a, 0x16, 0x4d, 0x69, 0x67, 0x72, - 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 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, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6c, + 0x6f, 0x67, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, + 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 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, 0xff, 0x02, + 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, + 0x65, 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, 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, 0x29, 0x0a, 0x06, 0x76, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x76, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x06, 0x76, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x12, 0x42, 0x0a, 0x1e, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, + 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x77, 0x69, 0x74, 0x68, + 0x5f, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x1a, 0x63, 0x6f, + 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x65, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 0x57, + 0x69, 0x74, 0x68, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x06, 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, 0x07, 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, 0x22, + 0x1c, 0x0a, 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x77, 0x0a, + 0x1e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 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, 0x12, 0x0a, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, + 0x25, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x22, 0x4c, 0x0a, 0x1f, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, + 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x22, 0x56, 0x0a, 0x18, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x3a, 0x0a, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, + 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, + 0x67, 0x73, 0x52, 0x08, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x22, 0x1b, 0x0a, 0x19, + 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xdd, 0x05, 0x0a, 0x14, 0x4d, 0x69, + 0x67, 0x72, 0x61, 0x74, 0x65, 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, 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, 0x5b, 0x0a, 0x17, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 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, 0x85, 0x01, - 0x0a, 0x14, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x73, 0x65, 0x72, 0x76, - 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x6f, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x72, 0x6f, 0x6f, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, 0x52, 0x6f, 0x6f, - 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, - 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x2c, - 0x0a, 0x16, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x19, 0x0a, 0x17, - 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x26, 0x0a, 0x10, 0x4d, 0x6f, 0x75, 0x6e, 0x74, - 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, - 0x82, 0x01, 0x0a, 0x11, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, 0x54, 0x79, - 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x6f, 0x53, 0x65, 0x72, - 0x76, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x72, 0x6f, 0x6f, 0x74, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, 0x52, 0x6f, 0x6f, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x12, 0x0a, 0x10, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x4c, 0x69, 0x73, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x29, 0x0a, 0x11, 0x4d, 0x6f, 0x75, 0x6e, - 0x74, 0x4c, 0x69, 0x73, 0x74, 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, 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, 0x81, 0x02, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, - 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 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, + 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, 0x1d, 0x0a, 0x0a, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x05, 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, 0x06, 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, 0x07, 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, 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, 0x28, 0x0a, 0x10, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x18, 0x0b, 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, 0x0c, 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, 0x0d, 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, 0x0e, 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, 0x0f, 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, 0x10, + 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, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x6e, 0x6f, 0x52, 0x6f, 0x75, + 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x22, 0xe6, 0x01, 0x0a, 0x16, 0x4d, 0x69, + 0x67, 0x72, 0x61, 0x74, 0x65, 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, 0x5b, 0x0a, 0x17, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 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, + 0x85, 0x01, 0x0a, 0x14, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, + 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, + 0x6f, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x6f, + 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x72, + 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, 0x52, + 0x6f, 0x6f, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4d, 0x6f, 0x75, 0x6e, 0x74, + 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x2c, 0x0a, 0x16, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, + 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x19, + 0x0a, 0x17, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x26, 0x0a, 0x10, 0x4d, 0x6f, 0x75, + 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x22, 0x82, 0x01, 0x0a, 0x11, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x73, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x6f, 0x70, 0x6f, 0x53, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x70, 0x6f, 0x5f, 0x72, 0x6f, + 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x70, 0x6f, 0x52, 0x6f, + 0x6f, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x12, 0x0a, 0x10, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x4c, + 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x29, 0x0a, 0x11, 0x4d, 0x6f, + 0x75, 0x6e, 0x74, 0x4c, 0x69, 0x73, 0x74, 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, 0x82, 0x07, 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, 0x12, 0x45, 0x0a, 0x10, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x81, 0x02, 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, 0x12, 0x16, + 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 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, 0x22, 0x14, 0x0a, 0x12, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd7, 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, 0x12, 0x4c, 0x0a, 0x19, 0x74, 0x6f, 0x6c, 0x65, 0x72, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, + 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x17, 0x74, 0x6f, + 0x6c, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x4c, 0x61, 0x67, 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, 0x14, 0x0a, 0x12, - 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0xd7, 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, + 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, 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, 0x12, 0x4c, - 0x0a, 0x19, 0x74, 0x6f, 0x6c, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x17, 0x74, 0x6f, 0x6c, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 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, + 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, + 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, 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, + 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, 0x05, 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, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x69, 0x6e, 0x63, 0x6c, + 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, 0x04, 0x20, 0x01, 0x28, 0x05, 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, 0x05, 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, 0x8f, 0x04, 0x0a, 0x14, 0x52, - 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 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, - 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, 0x23, 0x0a, 0x0d, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, - 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, - 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x05, - 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, 0x06, 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, 0x07, 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, 0x28, 0x0a, 0x10, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x73, 0x6b, - 0x69, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x15, 0x0a, 0x06, - 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x09, 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, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x74, - 0x6f, 0x70, 0x41, 0x66, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x70, 0x79, 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, 0x0b, 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, 0x0c, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x53, 0x74, 0x61, 0x72, 0x74, 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, + 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 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, - 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, 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, 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, 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, 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, + 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, 0x8f, 0x04, 0x0a, 0x14, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 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, 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, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x73, + 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0c, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x14, + 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x05, 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, 0x06, 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, 0x07, 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, 0x28, 0x0a, 0x10, 0x73, + 0x6b, 0x69, 0x70, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x73, 0x6b, 0x69, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x15, 0x0a, 0x06, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, + 0x09, 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, + 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x41, 0x66, 0x74, 0x65, 0x72, + 0x43, 0x6f, 0x70, 0x79, 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, 0x0b, 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, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, + 0x53, 0x74, 0x61, 0x72, 0x74, 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, 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, + 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, 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, 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, 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, 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, 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, + 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, 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, 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, + 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, 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, + 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, 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, + 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, 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, + 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, - 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, + 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, 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, 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, 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, + 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, 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, 0x88, 0x07, - 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 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, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, - 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, - 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x21, 0x0a, - 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, - 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, - 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, - 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, - 0x70, 0x65, 0x73, 0x18, 0x06, 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, 0x07, 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, 0x16, 0x0a, 0x06, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x55, 0x0a, 0x1e, 0x66, 0x69, 0x6c, 0x74, - 0x65, 0x72, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x1b, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, - 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x0b, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x12, 0x1a, 0x0a, 0x09, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x0c, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x4b, 0x73, 0x12, 0x2c, 0x0a, 0x12, - 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, - 0x74, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x38, 0x0a, 0x19, 0x6d, 0x61, - 0x78, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x5f, - 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6d, - 0x61, 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x72, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x77, 0x61, 0x69, 0x74, 0x18, 0x0f, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x04, 0x77, 0x61, 0x69, 0x74, 0x12, 0x42, 0x0a, 0x14, 0x77, 0x61, 0x69, 0x74, - 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, - 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, - 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x77, 0x61, 0x69, 0x74, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, - 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x76, - 0x65, 0x72, 0x62, 0x6f, 0x73, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x76, 0x65, - 0x72, 0x62, 0x6f, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x16, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, - 0x6f, 0x72, 0x74, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, - 0x13, 0x20, 0x01, 0x28, 0x03, 0x52, 0x13, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, - 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x3c, 0x0a, 0x11, 0x6d, 0x61, - 0x78, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x6d, 0x61, 0x78, 0x44, 0x69, 0x66, 0x66, - 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x29, 0x0a, 0x13, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x12, 0x0a, 0x04, 0x55, 0x55, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x55, - 0x55, 0x49, 0x44, 0x22, 0x6b, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 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, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, - 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x10, - 0x0a, 0x03, 0x61, 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x61, 0x72, 0x67, - 0x22, 0x15, 0x0a, 0x13, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6d, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, - 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, + 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, 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, 0x88, 0x07, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, + 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, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0x15, 0x0a, 0x13, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, - 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x69, 0x0a, - 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 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, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x72, 0x67, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x61, 0x72, 0x67, 0x22, 0xd7, 0x01, 0x0a, 0x11, 0x56, 0x44, 0x69, - 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, - 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x1a, 0x64, 0x0a, 0x14, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 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, 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x22, 0x6b, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, 0x6f, 0x70, 0x52, + 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x06, 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, 0x07, 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, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6c, + 0x69, 0x6d, 0x69, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, + 0x74, 0x12, 0x55, 0x0a, 0x1e, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x5f, 0x72, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, + 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x1b, 0x66, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x61, 0x69, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x62, 0x75, + 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x64, + 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x1a, 0x0a, 0x09, 0x6f, 0x6e, 0x6c, + 0x79, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, + 0x6c, 0x79, 0x50, 0x4b, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, + 0x61, 0x74, 0x73, 0x12, 0x38, 0x0a, 0x19, 0x6d, 0x61, 0x78, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, + 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6d, 0x61, 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, + 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x77, 0x61, 0x69, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x77, 0x61, 0x69, + 0x74, 0x12, 0x42, 0x0a, 0x14, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x12, 0x77, 0x61, 0x69, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, + 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, + 0x74, 0x72, 0x79, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, + 0x65, 0x74, 0x72, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x62, 0x6f, 0x73, 0x65, 0x18, + 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x76, 0x65, 0x72, 0x62, 0x6f, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x16, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x73, 0x61, 0x6d, + 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x13, 0x20, 0x01, 0x28, 0x03, 0x52, 0x13, + 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, + 0x6f, 0x77, 0x73, 0x12, 0x3c, 0x0a, 0x11, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, + 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x0f, 0x6d, 0x61, 0x78, 0x44, 0x69, 0x66, 0x66, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0x29, 0x0a, 0x13, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x55, 0x55, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x55, 0x55, 0x49, 0x44, 0x22, 0x6b, 0x0a, 0x12, + 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 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, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x61, 0x72, 0x67, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x61, 0x72, 0x67, 0x22, 0x15, 0x0a, 0x13, 0x56, 0x44, 0x69, + 0x66, 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x6d, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 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, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, - 0x13, 0x0a, 0x11, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb2, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 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, 0x67, 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, 0x12, 0x16, - 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, - 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x22, 0xe6, 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, 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, - 0x74, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x53, 0x74, 0x61, 0x74, - 0x65, 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, + 0x15, 0x0a, 0x13, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x69, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, + 0x68, 0x6f, 0x77, 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, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, + 0x10, 0x0a, 0x03, 0x61, 0x72, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x61, 0x72, + 0x67, 0x22, 0xd7, 0x01, 0x0a, 0x11, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x31, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, + 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x73, 0x1a, 0x64, 0x0a, 0x14, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 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, + 0x36, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6b, 0x0a, 0x10, 0x56, + 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, 0x6f, 0x70, 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, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, + 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x75, 0x75, 0x69, 0x64, 0x22, 0x13, 0x0a, 0x11, 0x56, 0x44, 0x69, 0x66, + 0x66, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb2, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, + 0x64, 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, 0x67, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x22, + 0xe6, 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, 0x53, 0x68, 0x61, 0x72, 0x64, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0xef, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, - 0x72, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, - 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, + 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, 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, 0x74, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, + 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x74, 0x72, + 0x61, 0x66, 0x66, 0x69, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 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, 0xef, 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, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 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 ( @@ -18151,7 +18467,7 @@ func file_vtctldata_proto_rawDescGZIP() []byte { } var file_vtctldata_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_vtctldata_proto_msgTypes = make([]protoimpl.MessageInfo, 275) +var file_vtctldata_proto_msgTypes = make([]protoimpl.MessageInfo, 280) var file_vtctldata_proto_goTypes = []interface{}{ (MaterializationIntent)(0), // 0: vtctldata.MaterializationIntent (QueryOrdering)(0), // 1: vtctldata.QueryOrdering @@ -18164,530 +18480,541 @@ var file_vtctldata_proto_goTypes = []interface{}{ (*Keyspace)(nil), // 8: vtctldata.Keyspace (*SchemaMigration)(nil), // 9: vtctldata.SchemaMigration (*Shard)(nil), // 10: vtctldata.Shard - (*Workflow)(nil), // 11: vtctldata.Workflow - (*AddCellInfoRequest)(nil), // 12: vtctldata.AddCellInfoRequest - (*AddCellInfoResponse)(nil), // 13: vtctldata.AddCellInfoResponse - (*AddCellsAliasRequest)(nil), // 14: vtctldata.AddCellsAliasRequest - (*AddCellsAliasResponse)(nil), // 15: vtctldata.AddCellsAliasResponse - (*ApplyRoutingRulesRequest)(nil), // 16: vtctldata.ApplyRoutingRulesRequest - (*ApplyRoutingRulesResponse)(nil), // 17: vtctldata.ApplyRoutingRulesResponse - (*ApplyShardRoutingRulesRequest)(nil), // 18: vtctldata.ApplyShardRoutingRulesRequest - (*ApplyShardRoutingRulesResponse)(nil), // 19: vtctldata.ApplyShardRoutingRulesResponse - (*ApplySchemaRequest)(nil), // 20: vtctldata.ApplySchemaRequest - (*ApplySchemaResponse)(nil), // 21: vtctldata.ApplySchemaResponse - (*ApplyVSchemaRequest)(nil), // 22: vtctldata.ApplyVSchemaRequest - (*ApplyVSchemaResponse)(nil), // 23: vtctldata.ApplyVSchemaResponse - (*BackupRequest)(nil), // 24: vtctldata.BackupRequest - (*BackupResponse)(nil), // 25: vtctldata.BackupResponse - (*BackupShardRequest)(nil), // 26: vtctldata.BackupShardRequest - (*CancelSchemaMigrationRequest)(nil), // 27: vtctldata.CancelSchemaMigrationRequest - (*CancelSchemaMigrationResponse)(nil), // 28: vtctldata.CancelSchemaMigrationResponse - (*ChangeTabletTypeRequest)(nil), // 29: vtctldata.ChangeTabletTypeRequest - (*ChangeTabletTypeResponse)(nil), // 30: vtctldata.ChangeTabletTypeResponse - (*CleanupSchemaMigrationRequest)(nil), // 31: vtctldata.CleanupSchemaMigrationRequest - (*CleanupSchemaMigrationResponse)(nil), // 32: vtctldata.CleanupSchemaMigrationResponse - (*CompleteSchemaMigrationRequest)(nil), // 33: vtctldata.CompleteSchemaMigrationRequest - (*CompleteSchemaMigrationResponse)(nil), // 34: vtctldata.CompleteSchemaMigrationResponse - (*CreateKeyspaceRequest)(nil), // 35: vtctldata.CreateKeyspaceRequest - (*CreateKeyspaceResponse)(nil), // 36: vtctldata.CreateKeyspaceResponse - (*CreateShardRequest)(nil), // 37: vtctldata.CreateShardRequest - (*CreateShardResponse)(nil), // 38: vtctldata.CreateShardResponse - (*DeleteCellInfoRequest)(nil), // 39: vtctldata.DeleteCellInfoRequest - (*DeleteCellInfoResponse)(nil), // 40: vtctldata.DeleteCellInfoResponse - (*DeleteCellsAliasRequest)(nil), // 41: vtctldata.DeleteCellsAliasRequest - (*DeleteCellsAliasResponse)(nil), // 42: vtctldata.DeleteCellsAliasResponse - (*DeleteKeyspaceRequest)(nil), // 43: vtctldata.DeleteKeyspaceRequest - (*DeleteKeyspaceResponse)(nil), // 44: vtctldata.DeleteKeyspaceResponse - (*DeleteShardsRequest)(nil), // 45: vtctldata.DeleteShardsRequest - (*DeleteShardsResponse)(nil), // 46: vtctldata.DeleteShardsResponse - (*DeleteSrvVSchemaRequest)(nil), // 47: vtctldata.DeleteSrvVSchemaRequest - (*DeleteSrvVSchemaResponse)(nil), // 48: vtctldata.DeleteSrvVSchemaResponse - (*DeleteTabletsRequest)(nil), // 49: vtctldata.DeleteTabletsRequest - (*DeleteTabletsResponse)(nil), // 50: vtctldata.DeleteTabletsResponse - (*EmergencyReparentShardRequest)(nil), // 51: vtctldata.EmergencyReparentShardRequest - (*EmergencyReparentShardResponse)(nil), // 52: vtctldata.EmergencyReparentShardResponse - (*ExecuteFetchAsAppRequest)(nil), // 53: vtctldata.ExecuteFetchAsAppRequest - (*ExecuteFetchAsAppResponse)(nil), // 54: vtctldata.ExecuteFetchAsAppResponse - (*ExecuteFetchAsDBARequest)(nil), // 55: vtctldata.ExecuteFetchAsDBARequest - (*ExecuteFetchAsDBAResponse)(nil), // 56: vtctldata.ExecuteFetchAsDBAResponse - (*ExecuteHookRequest)(nil), // 57: vtctldata.ExecuteHookRequest - (*ExecuteHookResponse)(nil), // 58: vtctldata.ExecuteHookResponse - (*ExecuteMultiFetchAsDBARequest)(nil), // 59: vtctldata.ExecuteMultiFetchAsDBARequest - (*ExecuteMultiFetchAsDBAResponse)(nil), // 60: vtctldata.ExecuteMultiFetchAsDBAResponse - (*FindAllShardsInKeyspaceRequest)(nil), // 61: vtctldata.FindAllShardsInKeyspaceRequest - (*FindAllShardsInKeyspaceResponse)(nil), // 62: vtctldata.FindAllShardsInKeyspaceResponse - (*ForceCutOverSchemaMigrationRequest)(nil), // 63: vtctldata.ForceCutOverSchemaMigrationRequest - (*ForceCutOverSchemaMigrationResponse)(nil), // 64: vtctldata.ForceCutOverSchemaMigrationResponse - (*GetBackupsRequest)(nil), // 65: vtctldata.GetBackupsRequest - (*GetBackupsResponse)(nil), // 66: vtctldata.GetBackupsResponse - (*GetCellInfoRequest)(nil), // 67: vtctldata.GetCellInfoRequest - (*GetCellInfoResponse)(nil), // 68: vtctldata.GetCellInfoResponse - (*GetCellInfoNamesRequest)(nil), // 69: vtctldata.GetCellInfoNamesRequest - (*GetCellInfoNamesResponse)(nil), // 70: vtctldata.GetCellInfoNamesResponse - (*GetCellsAliasesRequest)(nil), // 71: vtctldata.GetCellsAliasesRequest - (*GetCellsAliasesResponse)(nil), // 72: vtctldata.GetCellsAliasesResponse - (*GetFullStatusRequest)(nil), // 73: vtctldata.GetFullStatusRequest - (*GetFullStatusResponse)(nil), // 74: vtctldata.GetFullStatusResponse - (*GetKeyspacesRequest)(nil), // 75: vtctldata.GetKeyspacesRequest - (*GetKeyspacesResponse)(nil), // 76: vtctldata.GetKeyspacesResponse - (*GetKeyspaceRequest)(nil), // 77: vtctldata.GetKeyspaceRequest - (*GetKeyspaceResponse)(nil), // 78: vtctldata.GetKeyspaceResponse - (*GetPermissionsRequest)(nil), // 79: vtctldata.GetPermissionsRequest - (*GetPermissionsResponse)(nil), // 80: vtctldata.GetPermissionsResponse - (*GetRoutingRulesRequest)(nil), // 81: vtctldata.GetRoutingRulesRequest - (*GetRoutingRulesResponse)(nil), // 82: vtctldata.GetRoutingRulesResponse - (*GetSchemaRequest)(nil), // 83: vtctldata.GetSchemaRequest - (*GetSchemaResponse)(nil), // 84: vtctldata.GetSchemaResponse - (*GetSchemaMigrationsRequest)(nil), // 85: vtctldata.GetSchemaMigrationsRequest - (*GetSchemaMigrationsResponse)(nil), // 86: vtctldata.GetSchemaMigrationsResponse - (*GetShardReplicationRequest)(nil), // 87: vtctldata.GetShardReplicationRequest - (*GetShardReplicationResponse)(nil), // 88: vtctldata.GetShardReplicationResponse - (*GetShardRequest)(nil), // 89: vtctldata.GetShardRequest - (*GetShardResponse)(nil), // 90: vtctldata.GetShardResponse - (*GetShardRoutingRulesRequest)(nil), // 91: vtctldata.GetShardRoutingRulesRequest - (*GetShardRoutingRulesResponse)(nil), // 92: vtctldata.GetShardRoutingRulesResponse - (*GetSrvKeyspaceNamesRequest)(nil), // 93: vtctldata.GetSrvKeyspaceNamesRequest - (*GetSrvKeyspaceNamesResponse)(nil), // 94: vtctldata.GetSrvKeyspaceNamesResponse - (*GetSrvKeyspacesRequest)(nil), // 95: vtctldata.GetSrvKeyspacesRequest - (*GetSrvKeyspacesResponse)(nil), // 96: vtctldata.GetSrvKeyspacesResponse - (*UpdateThrottlerConfigRequest)(nil), // 97: vtctldata.UpdateThrottlerConfigRequest - (*UpdateThrottlerConfigResponse)(nil), // 98: vtctldata.UpdateThrottlerConfigResponse - (*GetSrvVSchemaRequest)(nil), // 99: vtctldata.GetSrvVSchemaRequest - (*GetSrvVSchemaResponse)(nil), // 100: vtctldata.GetSrvVSchemaResponse - (*GetSrvVSchemasRequest)(nil), // 101: vtctldata.GetSrvVSchemasRequest - (*GetSrvVSchemasResponse)(nil), // 102: vtctldata.GetSrvVSchemasResponse - (*GetTabletRequest)(nil), // 103: vtctldata.GetTabletRequest - (*GetTabletResponse)(nil), // 104: vtctldata.GetTabletResponse - (*GetTabletsRequest)(nil), // 105: vtctldata.GetTabletsRequest - (*GetTabletsResponse)(nil), // 106: vtctldata.GetTabletsResponse - (*GetTopologyPathRequest)(nil), // 107: vtctldata.GetTopologyPathRequest - (*GetTopologyPathResponse)(nil), // 108: vtctldata.GetTopologyPathResponse - (*TopologyCell)(nil), // 109: vtctldata.TopologyCell - (*GetVSchemaRequest)(nil), // 110: vtctldata.GetVSchemaRequest - (*GetVersionRequest)(nil), // 111: vtctldata.GetVersionRequest - (*GetVersionResponse)(nil), // 112: vtctldata.GetVersionResponse - (*GetVSchemaResponse)(nil), // 113: vtctldata.GetVSchemaResponse - (*GetWorkflowsRequest)(nil), // 114: vtctldata.GetWorkflowsRequest - (*GetWorkflowsResponse)(nil), // 115: vtctldata.GetWorkflowsResponse - (*InitShardPrimaryRequest)(nil), // 116: vtctldata.InitShardPrimaryRequest - (*InitShardPrimaryResponse)(nil), // 117: vtctldata.InitShardPrimaryResponse - (*LaunchSchemaMigrationRequest)(nil), // 118: vtctldata.LaunchSchemaMigrationRequest - (*LaunchSchemaMigrationResponse)(nil), // 119: vtctldata.LaunchSchemaMigrationResponse - (*LookupVindexCreateRequest)(nil), // 120: vtctldata.LookupVindexCreateRequest - (*LookupVindexCreateResponse)(nil), // 121: vtctldata.LookupVindexCreateResponse - (*LookupVindexExternalizeRequest)(nil), // 122: vtctldata.LookupVindexExternalizeRequest - (*LookupVindexExternalizeResponse)(nil), // 123: vtctldata.LookupVindexExternalizeResponse - (*MaterializeCreateRequest)(nil), // 124: vtctldata.MaterializeCreateRequest - (*MaterializeCreateResponse)(nil), // 125: vtctldata.MaterializeCreateResponse - (*MigrateCreateRequest)(nil), // 126: vtctldata.MigrateCreateRequest - (*MigrateCompleteRequest)(nil), // 127: vtctldata.MigrateCompleteRequest - (*MigrateCompleteResponse)(nil), // 128: vtctldata.MigrateCompleteResponse - (*MountRegisterRequest)(nil), // 129: vtctldata.MountRegisterRequest - (*MountRegisterResponse)(nil), // 130: vtctldata.MountRegisterResponse - (*MountUnregisterRequest)(nil), // 131: vtctldata.MountUnregisterRequest - (*MountUnregisterResponse)(nil), // 132: vtctldata.MountUnregisterResponse - (*MountShowRequest)(nil), // 133: vtctldata.MountShowRequest - (*MountShowResponse)(nil), // 134: vtctldata.MountShowResponse - (*MountListRequest)(nil), // 135: vtctldata.MountListRequest - (*MountListResponse)(nil), // 136: vtctldata.MountListResponse - (*MoveTablesCreateRequest)(nil), // 137: vtctldata.MoveTablesCreateRequest - (*MoveTablesCreateResponse)(nil), // 138: vtctldata.MoveTablesCreateResponse - (*MoveTablesCompleteRequest)(nil), // 139: vtctldata.MoveTablesCompleteRequest - (*MoveTablesCompleteResponse)(nil), // 140: vtctldata.MoveTablesCompleteResponse - (*PingTabletRequest)(nil), // 141: vtctldata.PingTabletRequest - (*PingTabletResponse)(nil), // 142: vtctldata.PingTabletResponse - (*PlannedReparentShardRequest)(nil), // 143: vtctldata.PlannedReparentShardRequest - (*PlannedReparentShardResponse)(nil), // 144: vtctldata.PlannedReparentShardResponse - (*RebuildKeyspaceGraphRequest)(nil), // 145: vtctldata.RebuildKeyspaceGraphRequest - (*RebuildKeyspaceGraphResponse)(nil), // 146: vtctldata.RebuildKeyspaceGraphResponse - (*RebuildVSchemaGraphRequest)(nil), // 147: vtctldata.RebuildVSchemaGraphRequest - (*RebuildVSchemaGraphResponse)(nil), // 148: vtctldata.RebuildVSchemaGraphResponse - (*RefreshStateRequest)(nil), // 149: vtctldata.RefreshStateRequest - (*RefreshStateResponse)(nil), // 150: vtctldata.RefreshStateResponse - (*RefreshStateByShardRequest)(nil), // 151: vtctldata.RefreshStateByShardRequest - (*RefreshStateByShardResponse)(nil), // 152: vtctldata.RefreshStateByShardResponse - (*ReloadSchemaRequest)(nil), // 153: vtctldata.ReloadSchemaRequest - (*ReloadSchemaResponse)(nil), // 154: vtctldata.ReloadSchemaResponse - (*ReloadSchemaKeyspaceRequest)(nil), // 155: vtctldata.ReloadSchemaKeyspaceRequest - (*ReloadSchemaKeyspaceResponse)(nil), // 156: vtctldata.ReloadSchemaKeyspaceResponse - (*ReloadSchemaShardRequest)(nil), // 157: vtctldata.ReloadSchemaShardRequest - (*ReloadSchemaShardResponse)(nil), // 158: vtctldata.ReloadSchemaShardResponse - (*RemoveBackupRequest)(nil), // 159: vtctldata.RemoveBackupRequest - (*RemoveBackupResponse)(nil), // 160: vtctldata.RemoveBackupResponse - (*RemoveKeyspaceCellRequest)(nil), // 161: vtctldata.RemoveKeyspaceCellRequest - (*RemoveKeyspaceCellResponse)(nil), // 162: vtctldata.RemoveKeyspaceCellResponse - (*RemoveShardCellRequest)(nil), // 163: vtctldata.RemoveShardCellRequest - (*RemoveShardCellResponse)(nil), // 164: vtctldata.RemoveShardCellResponse - (*ReparentTabletRequest)(nil), // 165: vtctldata.ReparentTabletRequest - (*ReparentTabletResponse)(nil), // 166: vtctldata.ReparentTabletResponse - (*ReshardCreateRequest)(nil), // 167: vtctldata.ReshardCreateRequest - (*RestoreFromBackupRequest)(nil), // 168: vtctldata.RestoreFromBackupRequest - (*RestoreFromBackupResponse)(nil), // 169: vtctldata.RestoreFromBackupResponse - (*RetrySchemaMigrationRequest)(nil), // 170: vtctldata.RetrySchemaMigrationRequest - (*RetrySchemaMigrationResponse)(nil), // 171: vtctldata.RetrySchemaMigrationResponse - (*RunHealthCheckRequest)(nil), // 172: vtctldata.RunHealthCheckRequest - (*RunHealthCheckResponse)(nil), // 173: vtctldata.RunHealthCheckResponse - (*SetKeyspaceDurabilityPolicyRequest)(nil), // 174: vtctldata.SetKeyspaceDurabilityPolicyRequest - (*SetKeyspaceDurabilityPolicyResponse)(nil), // 175: vtctldata.SetKeyspaceDurabilityPolicyResponse - (*SetKeyspaceShardingInfoRequest)(nil), // 176: vtctldata.SetKeyspaceShardingInfoRequest - (*SetKeyspaceShardingInfoResponse)(nil), // 177: vtctldata.SetKeyspaceShardingInfoResponse - (*SetShardIsPrimaryServingRequest)(nil), // 178: vtctldata.SetShardIsPrimaryServingRequest - (*SetShardIsPrimaryServingResponse)(nil), // 179: vtctldata.SetShardIsPrimaryServingResponse - (*SetShardTabletControlRequest)(nil), // 180: vtctldata.SetShardTabletControlRequest - (*SetShardTabletControlResponse)(nil), // 181: vtctldata.SetShardTabletControlResponse - (*SetWritableRequest)(nil), // 182: vtctldata.SetWritableRequest - (*SetWritableResponse)(nil), // 183: vtctldata.SetWritableResponse - (*ShardReplicationAddRequest)(nil), // 184: vtctldata.ShardReplicationAddRequest - (*ShardReplicationAddResponse)(nil), // 185: vtctldata.ShardReplicationAddResponse - (*ShardReplicationFixRequest)(nil), // 186: vtctldata.ShardReplicationFixRequest - (*ShardReplicationFixResponse)(nil), // 187: vtctldata.ShardReplicationFixResponse - (*ShardReplicationPositionsRequest)(nil), // 188: vtctldata.ShardReplicationPositionsRequest - (*ShardReplicationPositionsResponse)(nil), // 189: vtctldata.ShardReplicationPositionsResponse - (*ShardReplicationRemoveRequest)(nil), // 190: vtctldata.ShardReplicationRemoveRequest - (*ShardReplicationRemoveResponse)(nil), // 191: vtctldata.ShardReplicationRemoveResponse - (*SleepTabletRequest)(nil), // 192: vtctldata.SleepTabletRequest - (*SleepTabletResponse)(nil), // 193: vtctldata.SleepTabletResponse - (*SourceShardAddRequest)(nil), // 194: vtctldata.SourceShardAddRequest - (*SourceShardAddResponse)(nil), // 195: vtctldata.SourceShardAddResponse - (*SourceShardDeleteRequest)(nil), // 196: vtctldata.SourceShardDeleteRequest - (*SourceShardDeleteResponse)(nil), // 197: vtctldata.SourceShardDeleteResponse - (*StartReplicationRequest)(nil), // 198: vtctldata.StartReplicationRequest - (*StartReplicationResponse)(nil), // 199: vtctldata.StartReplicationResponse - (*StopReplicationRequest)(nil), // 200: vtctldata.StopReplicationRequest - (*StopReplicationResponse)(nil), // 201: vtctldata.StopReplicationResponse - (*TabletExternallyReparentedRequest)(nil), // 202: vtctldata.TabletExternallyReparentedRequest - (*TabletExternallyReparentedResponse)(nil), // 203: vtctldata.TabletExternallyReparentedResponse - (*UpdateCellInfoRequest)(nil), // 204: vtctldata.UpdateCellInfoRequest - (*UpdateCellInfoResponse)(nil), // 205: vtctldata.UpdateCellInfoResponse - (*UpdateCellsAliasRequest)(nil), // 206: vtctldata.UpdateCellsAliasRequest - (*UpdateCellsAliasResponse)(nil), // 207: vtctldata.UpdateCellsAliasResponse - (*ValidateRequest)(nil), // 208: vtctldata.ValidateRequest - (*ValidateResponse)(nil), // 209: vtctldata.ValidateResponse - (*ValidateKeyspaceRequest)(nil), // 210: vtctldata.ValidateKeyspaceRequest - (*ValidateKeyspaceResponse)(nil), // 211: vtctldata.ValidateKeyspaceResponse - (*ValidateSchemaKeyspaceRequest)(nil), // 212: vtctldata.ValidateSchemaKeyspaceRequest - (*ValidateSchemaKeyspaceResponse)(nil), // 213: vtctldata.ValidateSchemaKeyspaceResponse - (*ValidateShardRequest)(nil), // 214: vtctldata.ValidateShardRequest - (*ValidateShardResponse)(nil), // 215: vtctldata.ValidateShardResponse - (*ValidateVersionKeyspaceRequest)(nil), // 216: vtctldata.ValidateVersionKeyspaceRequest - (*ValidateVersionKeyspaceResponse)(nil), // 217: vtctldata.ValidateVersionKeyspaceResponse - (*ValidateVersionShardRequest)(nil), // 218: vtctldata.ValidateVersionShardRequest - (*ValidateVersionShardResponse)(nil), // 219: vtctldata.ValidateVersionShardResponse - (*ValidateVSchemaRequest)(nil), // 220: vtctldata.ValidateVSchemaRequest - (*ValidateVSchemaResponse)(nil), // 221: vtctldata.ValidateVSchemaResponse - (*VDiffCreateRequest)(nil), // 222: vtctldata.VDiffCreateRequest - (*VDiffCreateResponse)(nil), // 223: vtctldata.VDiffCreateResponse - (*VDiffDeleteRequest)(nil), // 224: vtctldata.VDiffDeleteRequest - (*VDiffDeleteResponse)(nil), // 225: vtctldata.VDiffDeleteResponse - (*VDiffResumeRequest)(nil), // 226: vtctldata.VDiffResumeRequest - (*VDiffResumeResponse)(nil), // 227: vtctldata.VDiffResumeResponse - (*VDiffShowRequest)(nil), // 228: vtctldata.VDiffShowRequest - (*VDiffShowResponse)(nil), // 229: vtctldata.VDiffShowResponse - (*VDiffStopRequest)(nil), // 230: vtctldata.VDiffStopRequest - (*VDiffStopResponse)(nil), // 231: vtctldata.VDiffStopResponse - (*WorkflowDeleteRequest)(nil), // 232: vtctldata.WorkflowDeleteRequest - (*WorkflowDeleteResponse)(nil), // 233: vtctldata.WorkflowDeleteResponse - (*WorkflowStatusRequest)(nil), // 234: vtctldata.WorkflowStatusRequest - (*WorkflowStatusResponse)(nil), // 235: vtctldata.WorkflowStatusResponse - (*WorkflowSwitchTrafficRequest)(nil), // 236: vtctldata.WorkflowSwitchTrafficRequest - (*WorkflowSwitchTrafficResponse)(nil), // 237: vtctldata.WorkflowSwitchTrafficResponse - (*WorkflowUpdateRequest)(nil), // 238: vtctldata.WorkflowUpdateRequest - (*WorkflowUpdateResponse)(nil), // 239: vtctldata.WorkflowUpdateResponse - nil, // 240: vtctldata.Workflow.ShardStreamsEntry - (*Workflow_ReplicationLocation)(nil), // 241: vtctldata.Workflow.ReplicationLocation - (*Workflow_ShardStream)(nil), // 242: vtctldata.Workflow.ShardStream - (*Workflow_Stream)(nil), // 243: vtctldata.Workflow.Stream - (*Workflow_Stream_CopyState)(nil), // 244: vtctldata.Workflow.Stream.CopyState - (*Workflow_Stream_Log)(nil), // 245: vtctldata.Workflow.Stream.Log - (*Workflow_Stream_ThrottlerStatus)(nil), // 246: vtctldata.Workflow.Stream.ThrottlerStatus - nil, // 247: vtctldata.ApplySchemaResponse.RowsAffectedByShardEntry - nil, // 248: vtctldata.ApplyVSchemaResponse.UnknownVindexParamsEntry - (*ApplyVSchemaResponse_ParamList)(nil), // 249: vtctldata.ApplyVSchemaResponse.ParamList - nil, // 250: vtctldata.CancelSchemaMigrationResponse.RowsAffectedByShardEntry - nil, // 251: vtctldata.CleanupSchemaMigrationResponse.RowsAffectedByShardEntry - nil, // 252: vtctldata.CompleteSchemaMigrationResponse.RowsAffectedByShardEntry - nil, // 253: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry - nil, // 254: vtctldata.ForceCutOverSchemaMigrationResponse.RowsAffectedByShardEntry - nil, // 255: vtctldata.GetCellsAliasesResponse.AliasesEntry - nil, // 256: vtctldata.GetShardReplicationResponse.ShardReplicationByCellEntry - nil, // 257: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry - (*GetSrvKeyspaceNamesResponse_NameList)(nil), // 258: vtctldata.GetSrvKeyspaceNamesResponse.NameList - nil, // 259: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry - nil, // 260: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry - nil, // 261: vtctldata.LaunchSchemaMigrationResponse.RowsAffectedByShardEntry - (*MoveTablesCreateResponse_TabletInfo)(nil), // 262: vtctldata.MoveTablesCreateResponse.TabletInfo - nil, // 263: vtctldata.RetrySchemaMigrationResponse.RowsAffectedByShardEntry - nil, // 264: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry - nil, // 265: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry - nil, // 266: vtctldata.ValidateResponse.ResultsByKeyspaceEntry - nil, // 267: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry - nil, // 268: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry - nil, // 269: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry - nil, // 270: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry - nil, // 271: vtctldata.VDiffShowResponse.TabletResponsesEntry - (*WorkflowDeleteResponse_TabletInfo)(nil), // 272: vtctldata.WorkflowDeleteResponse.TabletInfo - (*WorkflowStatusResponse_TableCopyState)(nil), // 273: vtctldata.WorkflowStatusResponse.TableCopyState - (*WorkflowStatusResponse_ShardStreamState)(nil), // 274: vtctldata.WorkflowStatusResponse.ShardStreamState - (*WorkflowStatusResponse_ShardStreams)(nil), // 275: vtctldata.WorkflowStatusResponse.ShardStreams - nil, // 276: vtctldata.WorkflowStatusResponse.TableCopyStateEntry - nil, // 277: vtctldata.WorkflowStatusResponse.ShardStreamsEntry - (*WorkflowUpdateResponse_TabletInfo)(nil), // 278: vtctldata.WorkflowUpdateResponse.TabletInfo - (*logutil.Event)(nil), // 279: logutil.Event - (tabletmanagerdata.TabletSelectionPreference)(0), // 280: tabletmanagerdata.TabletSelectionPreference - (*topodata.Keyspace)(nil), // 281: topodata.Keyspace - (*vttime.Time)(nil), // 282: vttime.Time - (*topodata.TabletAlias)(nil), // 283: topodata.TabletAlias - (*vttime.Duration)(nil), // 284: vttime.Duration - (*topodata.Shard)(nil), // 285: topodata.Shard - (*topodata.CellInfo)(nil), // 286: topodata.CellInfo - (*vschema.RoutingRules)(nil), // 287: vschema.RoutingRules - (*vschema.ShardRoutingRules)(nil), // 288: vschema.ShardRoutingRules - (*vtrpc.CallerID)(nil), // 289: vtrpc.CallerID - (*vschema.Keyspace)(nil), // 290: vschema.Keyspace - (topodata.TabletType)(0), // 291: topodata.TabletType - (*topodata.Tablet)(nil), // 292: topodata.Tablet - (topodata.KeyspaceType)(0), // 293: topodata.KeyspaceType - (*query.QueryResult)(nil), // 294: query.QueryResult - (*tabletmanagerdata.ExecuteHookRequest)(nil), // 295: tabletmanagerdata.ExecuteHookRequest - (*tabletmanagerdata.ExecuteHookResponse)(nil), // 296: tabletmanagerdata.ExecuteHookResponse - (*mysqlctl.BackupInfo)(nil), // 297: mysqlctl.BackupInfo - (*replicationdata.FullStatus)(nil), // 298: replicationdata.FullStatus - (*tabletmanagerdata.Permissions)(nil), // 299: tabletmanagerdata.Permissions - (*tabletmanagerdata.SchemaDefinition)(nil), // 300: tabletmanagerdata.SchemaDefinition - (*topodata.ThrottledAppRule)(nil), // 301: topodata.ThrottledAppRule - (*vschema.SrvVSchema)(nil), // 302: vschema.SrvVSchema - (*topodata.ShardReplicationError)(nil), // 303: topodata.ShardReplicationError - (*topodata.KeyRange)(nil), // 304: topodata.KeyRange - (*topodata.CellsAlias)(nil), // 305: topodata.CellsAlias - (*tabletmanagerdata.UpdateVReplicationWorkflowRequest)(nil), // 306: tabletmanagerdata.UpdateVReplicationWorkflowRequest - (*topodata.Shard_TabletControl)(nil), // 307: topodata.Shard.TabletControl - (*binlogdata.BinlogSource)(nil), // 308: binlogdata.BinlogSource - (*topodata.ShardReplication)(nil), // 309: topodata.ShardReplication - (*topodata.SrvKeyspace)(nil), // 310: topodata.SrvKeyspace - (*replicationdata.Status)(nil), // 311: replicationdata.Status - (*tabletmanagerdata.VDiffResponse)(nil), // 312: tabletmanagerdata.VDiffResponse + (*WorkflowOptions)(nil), // 11: vtctldata.WorkflowOptions + (*Workflow)(nil), // 12: vtctldata.Workflow + (*AddCellInfoRequest)(nil), // 13: vtctldata.AddCellInfoRequest + (*AddCellInfoResponse)(nil), // 14: vtctldata.AddCellInfoResponse + (*AddCellsAliasRequest)(nil), // 15: vtctldata.AddCellsAliasRequest + (*AddCellsAliasResponse)(nil), // 16: vtctldata.AddCellsAliasResponse + (*ApplyKeyspaceRoutingRulesRequest)(nil), // 17: vtctldata.ApplyKeyspaceRoutingRulesRequest + (*ApplyKeyspaceRoutingRulesResponse)(nil), // 18: vtctldata.ApplyKeyspaceRoutingRulesResponse + (*ApplyRoutingRulesRequest)(nil), // 19: vtctldata.ApplyRoutingRulesRequest + (*ApplyRoutingRulesResponse)(nil), // 20: vtctldata.ApplyRoutingRulesResponse + (*ApplyShardRoutingRulesRequest)(nil), // 21: vtctldata.ApplyShardRoutingRulesRequest + (*ApplyShardRoutingRulesResponse)(nil), // 22: vtctldata.ApplyShardRoutingRulesResponse + (*ApplySchemaRequest)(nil), // 23: vtctldata.ApplySchemaRequest + (*ApplySchemaResponse)(nil), // 24: vtctldata.ApplySchemaResponse + (*ApplyVSchemaRequest)(nil), // 25: vtctldata.ApplyVSchemaRequest + (*ApplyVSchemaResponse)(nil), // 26: vtctldata.ApplyVSchemaResponse + (*BackupRequest)(nil), // 27: vtctldata.BackupRequest + (*BackupResponse)(nil), // 28: vtctldata.BackupResponse + (*BackupShardRequest)(nil), // 29: vtctldata.BackupShardRequest + (*CancelSchemaMigrationRequest)(nil), // 30: vtctldata.CancelSchemaMigrationRequest + (*CancelSchemaMigrationResponse)(nil), // 31: vtctldata.CancelSchemaMigrationResponse + (*ChangeTabletTypeRequest)(nil), // 32: vtctldata.ChangeTabletTypeRequest + (*ChangeTabletTypeResponse)(nil), // 33: vtctldata.ChangeTabletTypeResponse + (*CleanupSchemaMigrationRequest)(nil), // 34: vtctldata.CleanupSchemaMigrationRequest + (*CleanupSchemaMigrationResponse)(nil), // 35: vtctldata.CleanupSchemaMigrationResponse + (*CompleteSchemaMigrationRequest)(nil), // 36: vtctldata.CompleteSchemaMigrationRequest + (*CompleteSchemaMigrationResponse)(nil), // 37: vtctldata.CompleteSchemaMigrationResponse + (*CreateKeyspaceRequest)(nil), // 38: vtctldata.CreateKeyspaceRequest + (*CreateKeyspaceResponse)(nil), // 39: vtctldata.CreateKeyspaceResponse + (*CreateShardRequest)(nil), // 40: vtctldata.CreateShardRequest + (*CreateShardResponse)(nil), // 41: vtctldata.CreateShardResponse + (*DeleteCellInfoRequest)(nil), // 42: vtctldata.DeleteCellInfoRequest + (*DeleteCellInfoResponse)(nil), // 43: vtctldata.DeleteCellInfoResponse + (*DeleteCellsAliasRequest)(nil), // 44: vtctldata.DeleteCellsAliasRequest + (*DeleteCellsAliasResponse)(nil), // 45: vtctldata.DeleteCellsAliasResponse + (*DeleteKeyspaceRequest)(nil), // 46: vtctldata.DeleteKeyspaceRequest + (*DeleteKeyspaceResponse)(nil), // 47: vtctldata.DeleteKeyspaceResponse + (*DeleteShardsRequest)(nil), // 48: vtctldata.DeleteShardsRequest + (*DeleteShardsResponse)(nil), // 49: vtctldata.DeleteShardsResponse + (*DeleteSrvVSchemaRequest)(nil), // 50: vtctldata.DeleteSrvVSchemaRequest + (*DeleteSrvVSchemaResponse)(nil), // 51: vtctldata.DeleteSrvVSchemaResponse + (*DeleteTabletsRequest)(nil), // 52: vtctldata.DeleteTabletsRequest + (*DeleteTabletsResponse)(nil), // 53: vtctldata.DeleteTabletsResponse + (*EmergencyReparentShardRequest)(nil), // 54: vtctldata.EmergencyReparentShardRequest + (*EmergencyReparentShardResponse)(nil), // 55: vtctldata.EmergencyReparentShardResponse + (*ExecuteFetchAsAppRequest)(nil), // 56: vtctldata.ExecuteFetchAsAppRequest + (*ExecuteFetchAsAppResponse)(nil), // 57: vtctldata.ExecuteFetchAsAppResponse + (*ExecuteFetchAsDBARequest)(nil), // 58: vtctldata.ExecuteFetchAsDBARequest + (*ExecuteFetchAsDBAResponse)(nil), // 59: vtctldata.ExecuteFetchAsDBAResponse + (*ExecuteHookRequest)(nil), // 60: vtctldata.ExecuteHookRequest + (*ExecuteHookResponse)(nil), // 61: vtctldata.ExecuteHookResponse + (*ExecuteMultiFetchAsDBARequest)(nil), // 62: vtctldata.ExecuteMultiFetchAsDBARequest + (*ExecuteMultiFetchAsDBAResponse)(nil), // 63: vtctldata.ExecuteMultiFetchAsDBAResponse + (*FindAllShardsInKeyspaceRequest)(nil), // 64: vtctldata.FindAllShardsInKeyspaceRequest + (*FindAllShardsInKeyspaceResponse)(nil), // 65: vtctldata.FindAllShardsInKeyspaceResponse + (*ForceCutOverSchemaMigrationRequest)(nil), // 66: vtctldata.ForceCutOverSchemaMigrationRequest + (*ForceCutOverSchemaMigrationResponse)(nil), // 67: vtctldata.ForceCutOverSchemaMigrationResponse + (*GetBackupsRequest)(nil), // 68: vtctldata.GetBackupsRequest + (*GetBackupsResponse)(nil), // 69: vtctldata.GetBackupsResponse + (*GetCellInfoRequest)(nil), // 70: vtctldata.GetCellInfoRequest + (*GetCellInfoResponse)(nil), // 71: vtctldata.GetCellInfoResponse + (*GetCellInfoNamesRequest)(nil), // 72: vtctldata.GetCellInfoNamesRequest + (*GetCellInfoNamesResponse)(nil), // 73: vtctldata.GetCellInfoNamesResponse + (*GetCellsAliasesRequest)(nil), // 74: vtctldata.GetCellsAliasesRequest + (*GetCellsAliasesResponse)(nil), // 75: vtctldata.GetCellsAliasesResponse + (*GetFullStatusRequest)(nil), // 76: vtctldata.GetFullStatusRequest + (*GetFullStatusResponse)(nil), // 77: vtctldata.GetFullStatusResponse + (*GetKeyspacesRequest)(nil), // 78: vtctldata.GetKeyspacesRequest + (*GetKeyspacesResponse)(nil), // 79: vtctldata.GetKeyspacesResponse + (*GetKeyspaceRequest)(nil), // 80: vtctldata.GetKeyspaceRequest + (*GetKeyspaceResponse)(nil), // 81: vtctldata.GetKeyspaceResponse + (*GetPermissionsRequest)(nil), // 82: vtctldata.GetPermissionsRequest + (*GetPermissionsResponse)(nil), // 83: vtctldata.GetPermissionsResponse + (*GetKeyspaceRoutingRulesRequest)(nil), // 84: vtctldata.GetKeyspaceRoutingRulesRequest + (*GetKeyspaceRoutingRulesResponse)(nil), // 85: vtctldata.GetKeyspaceRoutingRulesResponse + (*GetRoutingRulesRequest)(nil), // 86: vtctldata.GetRoutingRulesRequest + (*GetRoutingRulesResponse)(nil), // 87: vtctldata.GetRoutingRulesResponse + (*GetSchemaRequest)(nil), // 88: vtctldata.GetSchemaRequest + (*GetSchemaResponse)(nil), // 89: vtctldata.GetSchemaResponse + (*GetSchemaMigrationsRequest)(nil), // 90: vtctldata.GetSchemaMigrationsRequest + (*GetSchemaMigrationsResponse)(nil), // 91: vtctldata.GetSchemaMigrationsResponse + (*GetShardReplicationRequest)(nil), // 92: vtctldata.GetShardReplicationRequest + (*GetShardReplicationResponse)(nil), // 93: vtctldata.GetShardReplicationResponse + (*GetShardRequest)(nil), // 94: vtctldata.GetShardRequest + (*GetShardResponse)(nil), // 95: vtctldata.GetShardResponse + (*GetShardRoutingRulesRequest)(nil), // 96: vtctldata.GetShardRoutingRulesRequest + (*GetShardRoutingRulesResponse)(nil), // 97: vtctldata.GetShardRoutingRulesResponse + (*GetSrvKeyspaceNamesRequest)(nil), // 98: vtctldata.GetSrvKeyspaceNamesRequest + (*GetSrvKeyspaceNamesResponse)(nil), // 99: vtctldata.GetSrvKeyspaceNamesResponse + (*GetSrvKeyspacesRequest)(nil), // 100: vtctldata.GetSrvKeyspacesRequest + (*GetSrvKeyspacesResponse)(nil), // 101: vtctldata.GetSrvKeyspacesResponse + (*UpdateThrottlerConfigRequest)(nil), // 102: vtctldata.UpdateThrottlerConfigRequest + (*UpdateThrottlerConfigResponse)(nil), // 103: vtctldata.UpdateThrottlerConfigResponse + (*GetSrvVSchemaRequest)(nil), // 104: vtctldata.GetSrvVSchemaRequest + (*GetSrvVSchemaResponse)(nil), // 105: vtctldata.GetSrvVSchemaResponse + (*GetSrvVSchemasRequest)(nil), // 106: vtctldata.GetSrvVSchemasRequest + (*GetSrvVSchemasResponse)(nil), // 107: vtctldata.GetSrvVSchemasResponse + (*GetTabletRequest)(nil), // 108: vtctldata.GetTabletRequest + (*GetTabletResponse)(nil), // 109: vtctldata.GetTabletResponse + (*GetTabletsRequest)(nil), // 110: vtctldata.GetTabletsRequest + (*GetTabletsResponse)(nil), // 111: vtctldata.GetTabletsResponse + (*GetTopologyPathRequest)(nil), // 112: vtctldata.GetTopologyPathRequest + (*GetTopologyPathResponse)(nil), // 113: vtctldata.GetTopologyPathResponse + (*TopologyCell)(nil), // 114: vtctldata.TopologyCell + (*GetVSchemaRequest)(nil), // 115: vtctldata.GetVSchemaRequest + (*GetVersionRequest)(nil), // 116: vtctldata.GetVersionRequest + (*GetVersionResponse)(nil), // 117: vtctldata.GetVersionResponse + (*GetVSchemaResponse)(nil), // 118: vtctldata.GetVSchemaResponse + (*GetWorkflowsRequest)(nil), // 119: vtctldata.GetWorkflowsRequest + (*GetWorkflowsResponse)(nil), // 120: vtctldata.GetWorkflowsResponse + (*InitShardPrimaryRequest)(nil), // 121: vtctldata.InitShardPrimaryRequest + (*InitShardPrimaryResponse)(nil), // 122: vtctldata.InitShardPrimaryResponse + (*LaunchSchemaMigrationRequest)(nil), // 123: vtctldata.LaunchSchemaMigrationRequest + (*LaunchSchemaMigrationResponse)(nil), // 124: vtctldata.LaunchSchemaMigrationResponse + (*LookupVindexCreateRequest)(nil), // 125: vtctldata.LookupVindexCreateRequest + (*LookupVindexCreateResponse)(nil), // 126: vtctldata.LookupVindexCreateResponse + (*LookupVindexExternalizeRequest)(nil), // 127: vtctldata.LookupVindexExternalizeRequest + (*LookupVindexExternalizeResponse)(nil), // 128: vtctldata.LookupVindexExternalizeResponse + (*MaterializeCreateRequest)(nil), // 129: vtctldata.MaterializeCreateRequest + (*MaterializeCreateResponse)(nil), // 130: vtctldata.MaterializeCreateResponse + (*MigrateCreateRequest)(nil), // 131: vtctldata.MigrateCreateRequest + (*MigrateCompleteRequest)(nil), // 132: vtctldata.MigrateCompleteRequest + (*MigrateCompleteResponse)(nil), // 133: vtctldata.MigrateCompleteResponse + (*MountRegisterRequest)(nil), // 134: vtctldata.MountRegisterRequest + (*MountRegisterResponse)(nil), // 135: vtctldata.MountRegisterResponse + (*MountUnregisterRequest)(nil), // 136: vtctldata.MountUnregisterRequest + (*MountUnregisterResponse)(nil), // 137: vtctldata.MountUnregisterResponse + (*MountShowRequest)(nil), // 138: vtctldata.MountShowRequest + (*MountShowResponse)(nil), // 139: vtctldata.MountShowResponse + (*MountListRequest)(nil), // 140: vtctldata.MountListRequest + (*MountListResponse)(nil), // 141: vtctldata.MountListResponse + (*MoveTablesCreateRequest)(nil), // 142: vtctldata.MoveTablesCreateRequest + (*MoveTablesCreateResponse)(nil), // 143: vtctldata.MoveTablesCreateResponse + (*MoveTablesCompleteRequest)(nil), // 144: vtctldata.MoveTablesCompleteRequest + (*MoveTablesCompleteResponse)(nil), // 145: vtctldata.MoveTablesCompleteResponse + (*PingTabletRequest)(nil), // 146: vtctldata.PingTabletRequest + (*PingTabletResponse)(nil), // 147: vtctldata.PingTabletResponse + (*PlannedReparentShardRequest)(nil), // 148: vtctldata.PlannedReparentShardRequest + (*PlannedReparentShardResponse)(nil), // 149: vtctldata.PlannedReparentShardResponse + (*RebuildKeyspaceGraphRequest)(nil), // 150: vtctldata.RebuildKeyspaceGraphRequest + (*RebuildKeyspaceGraphResponse)(nil), // 151: vtctldata.RebuildKeyspaceGraphResponse + (*RebuildVSchemaGraphRequest)(nil), // 152: vtctldata.RebuildVSchemaGraphRequest + (*RebuildVSchemaGraphResponse)(nil), // 153: vtctldata.RebuildVSchemaGraphResponse + (*RefreshStateRequest)(nil), // 154: vtctldata.RefreshStateRequest + (*RefreshStateResponse)(nil), // 155: vtctldata.RefreshStateResponse + (*RefreshStateByShardRequest)(nil), // 156: vtctldata.RefreshStateByShardRequest + (*RefreshStateByShardResponse)(nil), // 157: vtctldata.RefreshStateByShardResponse + (*ReloadSchemaRequest)(nil), // 158: vtctldata.ReloadSchemaRequest + (*ReloadSchemaResponse)(nil), // 159: vtctldata.ReloadSchemaResponse + (*ReloadSchemaKeyspaceRequest)(nil), // 160: vtctldata.ReloadSchemaKeyspaceRequest + (*ReloadSchemaKeyspaceResponse)(nil), // 161: vtctldata.ReloadSchemaKeyspaceResponse + (*ReloadSchemaShardRequest)(nil), // 162: vtctldata.ReloadSchemaShardRequest + (*ReloadSchemaShardResponse)(nil), // 163: vtctldata.ReloadSchemaShardResponse + (*RemoveBackupRequest)(nil), // 164: vtctldata.RemoveBackupRequest + (*RemoveBackupResponse)(nil), // 165: vtctldata.RemoveBackupResponse + (*RemoveKeyspaceCellRequest)(nil), // 166: vtctldata.RemoveKeyspaceCellRequest + (*RemoveKeyspaceCellResponse)(nil), // 167: vtctldata.RemoveKeyspaceCellResponse + (*RemoveShardCellRequest)(nil), // 168: vtctldata.RemoveShardCellRequest + (*RemoveShardCellResponse)(nil), // 169: vtctldata.RemoveShardCellResponse + (*ReparentTabletRequest)(nil), // 170: vtctldata.ReparentTabletRequest + (*ReparentTabletResponse)(nil), // 171: vtctldata.ReparentTabletResponse + (*ReshardCreateRequest)(nil), // 172: vtctldata.ReshardCreateRequest + (*RestoreFromBackupRequest)(nil), // 173: vtctldata.RestoreFromBackupRequest + (*RestoreFromBackupResponse)(nil), // 174: vtctldata.RestoreFromBackupResponse + (*RetrySchemaMigrationRequest)(nil), // 175: vtctldata.RetrySchemaMigrationRequest + (*RetrySchemaMigrationResponse)(nil), // 176: vtctldata.RetrySchemaMigrationResponse + (*RunHealthCheckRequest)(nil), // 177: vtctldata.RunHealthCheckRequest + (*RunHealthCheckResponse)(nil), // 178: vtctldata.RunHealthCheckResponse + (*SetKeyspaceDurabilityPolicyRequest)(nil), // 179: vtctldata.SetKeyspaceDurabilityPolicyRequest + (*SetKeyspaceDurabilityPolicyResponse)(nil), // 180: vtctldata.SetKeyspaceDurabilityPolicyResponse + (*SetKeyspaceShardingInfoRequest)(nil), // 181: vtctldata.SetKeyspaceShardingInfoRequest + (*SetKeyspaceShardingInfoResponse)(nil), // 182: vtctldata.SetKeyspaceShardingInfoResponse + (*SetShardIsPrimaryServingRequest)(nil), // 183: vtctldata.SetShardIsPrimaryServingRequest + (*SetShardIsPrimaryServingResponse)(nil), // 184: vtctldata.SetShardIsPrimaryServingResponse + (*SetShardTabletControlRequest)(nil), // 185: vtctldata.SetShardTabletControlRequest + (*SetShardTabletControlResponse)(nil), // 186: vtctldata.SetShardTabletControlResponse + (*SetWritableRequest)(nil), // 187: vtctldata.SetWritableRequest + (*SetWritableResponse)(nil), // 188: vtctldata.SetWritableResponse + (*ShardReplicationAddRequest)(nil), // 189: vtctldata.ShardReplicationAddRequest + (*ShardReplicationAddResponse)(nil), // 190: vtctldata.ShardReplicationAddResponse + (*ShardReplicationFixRequest)(nil), // 191: vtctldata.ShardReplicationFixRequest + (*ShardReplicationFixResponse)(nil), // 192: vtctldata.ShardReplicationFixResponse + (*ShardReplicationPositionsRequest)(nil), // 193: vtctldata.ShardReplicationPositionsRequest + (*ShardReplicationPositionsResponse)(nil), // 194: vtctldata.ShardReplicationPositionsResponse + (*ShardReplicationRemoveRequest)(nil), // 195: vtctldata.ShardReplicationRemoveRequest + (*ShardReplicationRemoveResponse)(nil), // 196: vtctldata.ShardReplicationRemoveResponse + (*SleepTabletRequest)(nil), // 197: vtctldata.SleepTabletRequest + (*SleepTabletResponse)(nil), // 198: vtctldata.SleepTabletResponse + (*SourceShardAddRequest)(nil), // 199: vtctldata.SourceShardAddRequest + (*SourceShardAddResponse)(nil), // 200: vtctldata.SourceShardAddResponse + (*SourceShardDeleteRequest)(nil), // 201: vtctldata.SourceShardDeleteRequest + (*SourceShardDeleteResponse)(nil), // 202: vtctldata.SourceShardDeleteResponse + (*StartReplicationRequest)(nil), // 203: vtctldata.StartReplicationRequest + (*StartReplicationResponse)(nil), // 204: vtctldata.StartReplicationResponse + (*StopReplicationRequest)(nil), // 205: vtctldata.StopReplicationRequest + (*StopReplicationResponse)(nil), // 206: vtctldata.StopReplicationResponse + (*TabletExternallyReparentedRequest)(nil), // 207: vtctldata.TabletExternallyReparentedRequest + (*TabletExternallyReparentedResponse)(nil), // 208: vtctldata.TabletExternallyReparentedResponse + (*UpdateCellInfoRequest)(nil), // 209: vtctldata.UpdateCellInfoRequest + (*UpdateCellInfoResponse)(nil), // 210: vtctldata.UpdateCellInfoResponse + (*UpdateCellsAliasRequest)(nil), // 211: vtctldata.UpdateCellsAliasRequest + (*UpdateCellsAliasResponse)(nil), // 212: vtctldata.UpdateCellsAliasResponse + (*ValidateRequest)(nil), // 213: vtctldata.ValidateRequest + (*ValidateResponse)(nil), // 214: vtctldata.ValidateResponse + (*ValidateKeyspaceRequest)(nil), // 215: vtctldata.ValidateKeyspaceRequest + (*ValidateKeyspaceResponse)(nil), // 216: vtctldata.ValidateKeyspaceResponse + (*ValidateSchemaKeyspaceRequest)(nil), // 217: vtctldata.ValidateSchemaKeyspaceRequest + (*ValidateSchemaKeyspaceResponse)(nil), // 218: vtctldata.ValidateSchemaKeyspaceResponse + (*ValidateShardRequest)(nil), // 219: vtctldata.ValidateShardRequest + (*ValidateShardResponse)(nil), // 220: vtctldata.ValidateShardResponse + (*ValidateVersionKeyspaceRequest)(nil), // 221: vtctldata.ValidateVersionKeyspaceRequest + (*ValidateVersionKeyspaceResponse)(nil), // 222: vtctldata.ValidateVersionKeyspaceResponse + (*ValidateVersionShardRequest)(nil), // 223: vtctldata.ValidateVersionShardRequest + (*ValidateVersionShardResponse)(nil), // 224: vtctldata.ValidateVersionShardResponse + (*ValidateVSchemaRequest)(nil), // 225: vtctldata.ValidateVSchemaRequest + (*ValidateVSchemaResponse)(nil), // 226: vtctldata.ValidateVSchemaResponse + (*VDiffCreateRequest)(nil), // 227: vtctldata.VDiffCreateRequest + (*VDiffCreateResponse)(nil), // 228: vtctldata.VDiffCreateResponse + (*VDiffDeleteRequest)(nil), // 229: vtctldata.VDiffDeleteRequest + (*VDiffDeleteResponse)(nil), // 230: vtctldata.VDiffDeleteResponse + (*VDiffResumeRequest)(nil), // 231: vtctldata.VDiffResumeRequest + (*VDiffResumeResponse)(nil), // 232: vtctldata.VDiffResumeResponse + (*VDiffShowRequest)(nil), // 233: vtctldata.VDiffShowRequest + (*VDiffShowResponse)(nil), // 234: vtctldata.VDiffShowResponse + (*VDiffStopRequest)(nil), // 235: vtctldata.VDiffStopRequest + (*VDiffStopResponse)(nil), // 236: vtctldata.VDiffStopResponse + (*WorkflowDeleteRequest)(nil), // 237: vtctldata.WorkflowDeleteRequest + (*WorkflowDeleteResponse)(nil), // 238: vtctldata.WorkflowDeleteResponse + (*WorkflowStatusRequest)(nil), // 239: vtctldata.WorkflowStatusRequest + (*WorkflowStatusResponse)(nil), // 240: vtctldata.WorkflowStatusResponse + (*WorkflowSwitchTrafficRequest)(nil), // 241: vtctldata.WorkflowSwitchTrafficRequest + (*WorkflowSwitchTrafficResponse)(nil), // 242: vtctldata.WorkflowSwitchTrafficResponse + (*WorkflowUpdateRequest)(nil), // 243: vtctldata.WorkflowUpdateRequest + (*WorkflowUpdateResponse)(nil), // 244: vtctldata.WorkflowUpdateResponse + nil, // 245: vtctldata.Workflow.ShardStreamsEntry + (*Workflow_ReplicationLocation)(nil), // 246: vtctldata.Workflow.ReplicationLocation + (*Workflow_ShardStream)(nil), // 247: vtctldata.Workflow.ShardStream + (*Workflow_Stream)(nil), // 248: vtctldata.Workflow.Stream + (*Workflow_Stream_CopyState)(nil), // 249: vtctldata.Workflow.Stream.CopyState + (*Workflow_Stream_Log)(nil), // 250: vtctldata.Workflow.Stream.Log + (*Workflow_Stream_ThrottlerStatus)(nil), // 251: vtctldata.Workflow.Stream.ThrottlerStatus + nil, // 252: vtctldata.ApplySchemaResponse.RowsAffectedByShardEntry + nil, // 253: vtctldata.ApplyVSchemaResponse.UnknownVindexParamsEntry + (*ApplyVSchemaResponse_ParamList)(nil), // 254: vtctldata.ApplyVSchemaResponse.ParamList + nil, // 255: vtctldata.CancelSchemaMigrationResponse.RowsAffectedByShardEntry + nil, // 256: vtctldata.CleanupSchemaMigrationResponse.RowsAffectedByShardEntry + nil, // 257: vtctldata.CompleteSchemaMigrationResponse.RowsAffectedByShardEntry + nil, // 258: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry + nil, // 259: vtctldata.ForceCutOverSchemaMigrationResponse.RowsAffectedByShardEntry + nil, // 260: vtctldata.GetCellsAliasesResponse.AliasesEntry + nil, // 261: vtctldata.GetShardReplicationResponse.ShardReplicationByCellEntry + nil, // 262: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry + (*GetSrvKeyspaceNamesResponse_NameList)(nil), // 263: vtctldata.GetSrvKeyspaceNamesResponse.NameList + nil, // 264: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry + nil, // 265: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry + nil, // 266: vtctldata.LaunchSchemaMigrationResponse.RowsAffectedByShardEntry + (*MoveTablesCreateResponse_TabletInfo)(nil), // 267: vtctldata.MoveTablesCreateResponse.TabletInfo + nil, // 268: vtctldata.RetrySchemaMigrationResponse.RowsAffectedByShardEntry + nil, // 269: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry + nil, // 270: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry + nil, // 271: vtctldata.ValidateResponse.ResultsByKeyspaceEntry + nil, // 272: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry + nil, // 273: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry + nil, // 274: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry + nil, // 275: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry + nil, // 276: vtctldata.VDiffShowResponse.TabletResponsesEntry + (*WorkflowDeleteResponse_TabletInfo)(nil), // 277: vtctldata.WorkflowDeleteResponse.TabletInfo + (*WorkflowStatusResponse_TableCopyState)(nil), // 278: vtctldata.WorkflowStatusResponse.TableCopyState + (*WorkflowStatusResponse_ShardStreamState)(nil), // 279: vtctldata.WorkflowStatusResponse.ShardStreamState + (*WorkflowStatusResponse_ShardStreams)(nil), // 280: vtctldata.WorkflowStatusResponse.ShardStreams + nil, // 281: vtctldata.WorkflowStatusResponse.TableCopyStateEntry + nil, // 282: vtctldata.WorkflowStatusResponse.ShardStreamsEntry + (*WorkflowUpdateResponse_TabletInfo)(nil), // 283: vtctldata.WorkflowUpdateResponse.TabletInfo + (*logutil.Event)(nil), // 284: logutil.Event + (tabletmanagerdata.TabletSelectionPreference)(0), // 285: tabletmanagerdata.TabletSelectionPreference + (*topodata.Keyspace)(nil), // 286: topodata.Keyspace + (*vttime.Time)(nil), // 287: vttime.Time + (*topodata.TabletAlias)(nil), // 288: topodata.TabletAlias + (*vttime.Duration)(nil), // 289: vttime.Duration + (*topodata.Shard)(nil), // 290: topodata.Shard + (*topodata.CellInfo)(nil), // 291: topodata.CellInfo + (*vschema.KeyspaceRoutingRules)(nil), // 292: vschema.KeyspaceRoutingRules + (*vschema.RoutingRules)(nil), // 293: vschema.RoutingRules + (*vschema.ShardRoutingRules)(nil), // 294: vschema.ShardRoutingRules + (*vtrpc.CallerID)(nil), // 295: vtrpc.CallerID + (*vschema.Keyspace)(nil), // 296: vschema.Keyspace + (topodata.TabletType)(0), // 297: topodata.TabletType + (*topodata.Tablet)(nil), // 298: topodata.Tablet + (topodata.KeyspaceType)(0), // 299: topodata.KeyspaceType + (*query.QueryResult)(nil), // 300: query.QueryResult + (*tabletmanagerdata.ExecuteHookRequest)(nil), // 301: tabletmanagerdata.ExecuteHookRequest + (*tabletmanagerdata.ExecuteHookResponse)(nil), // 302: tabletmanagerdata.ExecuteHookResponse + (*mysqlctl.BackupInfo)(nil), // 303: mysqlctl.BackupInfo + (*replicationdata.FullStatus)(nil), // 304: replicationdata.FullStatus + (*tabletmanagerdata.Permissions)(nil), // 305: tabletmanagerdata.Permissions + (*tabletmanagerdata.SchemaDefinition)(nil), // 306: tabletmanagerdata.SchemaDefinition + (*topodata.ThrottledAppRule)(nil), // 307: topodata.ThrottledAppRule + (*vschema.SrvVSchema)(nil), // 308: vschema.SrvVSchema + (*topodata.ShardReplicationError)(nil), // 309: topodata.ShardReplicationError + (*topodata.KeyRange)(nil), // 310: topodata.KeyRange + (*topodata.CellsAlias)(nil), // 311: topodata.CellsAlias + (*tabletmanagerdata.UpdateVReplicationWorkflowRequest)(nil), // 312: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*topodata.Shard_TabletControl)(nil), // 313: topodata.Shard.TabletControl + (*binlogdata.BinlogSource)(nil), // 314: binlogdata.BinlogSource + (*topodata.ShardReplication)(nil), // 315: topodata.ShardReplication + (*topodata.SrvKeyspace)(nil), // 316: topodata.SrvKeyspace + (*replicationdata.Status)(nil), // 317: replicationdata.Status + (*tabletmanagerdata.VDiffResponse)(nil), // 318: tabletmanagerdata.VDiffResponse } var file_vtctldata_proto_depIdxs = []int32{ - 279, // 0: vtctldata.ExecuteVtctlCommandResponse.event:type_name -> logutil.Event + 284, // 0: vtctldata.ExecuteVtctlCommandResponse.event:type_name -> logutil.Event 6, // 1: vtctldata.MaterializeSettings.table_settings:type_name -> vtctldata.TableMaterializeSettings 0, // 2: vtctldata.MaterializeSettings.materialization_intent:type_name -> vtctldata.MaterializationIntent - 280, // 3: vtctldata.MaterializeSettings.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 281, // 4: vtctldata.Keyspace.keyspace:type_name -> topodata.Keyspace - 2, // 5: vtctldata.SchemaMigration.strategy:type_name -> vtctldata.SchemaMigration.Strategy - 282, // 6: vtctldata.SchemaMigration.added_at:type_name -> vttime.Time - 282, // 7: vtctldata.SchemaMigration.requested_at:type_name -> vttime.Time - 282, // 8: vtctldata.SchemaMigration.ready_at:type_name -> vttime.Time - 282, // 9: vtctldata.SchemaMigration.started_at:type_name -> vttime.Time - 282, // 10: vtctldata.SchemaMigration.liveness_timestamp:type_name -> vttime.Time - 282, // 11: vtctldata.SchemaMigration.completed_at:type_name -> vttime.Time - 282, // 12: vtctldata.SchemaMigration.cleaned_up_at:type_name -> vttime.Time - 3, // 13: vtctldata.SchemaMigration.status:type_name -> vtctldata.SchemaMigration.Status - 283, // 14: vtctldata.SchemaMigration.tablet:type_name -> topodata.TabletAlias - 284, // 15: vtctldata.SchemaMigration.artifact_retention:type_name -> vttime.Duration - 282, // 16: vtctldata.SchemaMigration.last_throttled_at:type_name -> vttime.Time - 282, // 17: vtctldata.SchemaMigration.cancelled_at:type_name -> vttime.Time - 282, // 18: vtctldata.SchemaMigration.reviewed_at:type_name -> vttime.Time - 282, // 19: vtctldata.SchemaMigration.ready_to_complete_at:type_name -> vttime.Time - 285, // 20: vtctldata.Shard.shard:type_name -> topodata.Shard - 241, // 21: vtctldata.Workflow.source:type_name -> vtctldata.Workflow.ReplicationLocation - 241, // 22: vtctldata.Workflow.target:type_name -> vtctldata.Workflow.ReplicationLocation - 240, // 23: vtctldata.Workflow.shard_streams:type_name -> vtctldata.Workflow.ShardStreamsEntry - 286, // 24: vtctldata.AddCellInfoRequest.cell_info:type_name -> topodata.CellInfo - 287, // 25: vtctldata.ApplyRoutingRulesRequest.routing_rules:type_name -> vschema.RoutingRules - 288, // 26: vtctldata.ApplyShardRoutingRulesRequest.shard_routing_rules:type_name -> vschema.ShardRoutingRules - 284, // 27: vtctldata.ApplySchemaRequest.wait_replicas_timeout:type_name -> vttime.Duration - 289, // 28: vtctldata.ApplySchemaRequest.caller_id:type_name -> vtrpc.CallerID - 247, // 29: vtctldata.ApplySchemaResponse.rows_affected_by_shard:type_name -> vtctldata.ApplySchemaResponse.RowsAffectedByShardEntry - 290, // 30: vtctldata.ApplyVSchemaRequest.v_schema:type_name -> vschema.Keyspace - 290, // 31: vtctldata.ApplyVSchemaResponse.v_schema:type_name -> vschema.Keyspace - 248, // 32: vtctldata.ApplyVSchemaResponse.unknown_vindex_params:type_name -> vtctldata.ApplyVSchemaResponse.UnknownVindexParamsEntry - 283, // 33: vtctldata.BackupRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 34: vtctldata.BackupResponse.tablet_alias:type_name -> topodata.TabletAlias - 279, // 35: vtctldata.BackupResponse.event:type_name -> logutil.Event - 250, // 36: vtctldata.CancelSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.CancelSchemaMigrationResponse.RowsAffectedByShardEntry - 283, // 37: vtctldata.ChangeTabletTypeRequest.tablet_alias:type_name -> topodata.TabletAlias - 291, // 38: vtctldata.ChangeTabletTypeRequest.db_type:type_name -> topodata.TabletType - 292, // 39: vtctldata.ChangeTabletTypeResponse.before_tablet:type_name -> topodata.Tablet - 292, // 40: vtctldata.ChangeTabletTypeResponse.after_tablet:type_name -> topodata.Tablet - 251, // 41: vtctldata.CleanupSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.CleanupSchemaMigrationResponse.RowsAffectedByShardEntry - 252, // 42: vtctldata.CompleteSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.CompleteSchemaMigrationResponse.RowsAffectedByShardEntry - 293, // 43: vtctldata.CreateKeyspaceRequest.type:type_name -> topodata.KeyspaceType - 282, // 44: vtctldata.CreateKeyspaceRequest.snapshot_time:type_name -> vttime.Time - 8, // 45: vtctldata.CreateKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace - 8, // 46: vtctldata.CreateShardResponse.keyspace:type_name -> vtctldata.Keyspace - 10, // 47: vtctldata.CreateShardResponse.shard:type_name -> vtctldata.Shard - 10, // 48: vtctldata.DeleteShardsRequest.shards:type_name -> vtctldata.Shard - 283, // 49: vtctldata.DeleteTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias - 283, // 50: vtctldata.EmergencyReparentShardRequest.new_primary:type_name -> topodata.TabletAlias - 283, // 51: vtctldata.EmergencyReparentShardRequest.ignore_replicas:type_name -> topodata.TabletAlias - 284, // 52: vtctldata.EmergencyReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration - 283, // 53: vtctldata.EmergencyReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias - 279, // 54: vtctldata.EmergencyReparentShardResponse.events:type_name -> logutil.Event - 283, // 55: vtctldata.ExecuteFetchAsAppRequest.tablet_alias:type_name -> topodata.TabletAlias - 294, // 56: vtctldata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult - 283, // 57: vtctldata.ExecuteFetchAsDBARequest.tablet_alias:type_name -> topodata.TabletAlias - 294, // 58: vtctldata.ExecuteFetchAsDBAResponse.result:type_name -> query.QueryResult - 283, // 59: vtctldata.ExecuteHookRequest.tablet_alias:type_name -> topodata.TabletAlias - 295, // 60: vtctldata.ExecuteHookRequest.tablet_hook_request:type_name -> tabletmanagerdata.ExecuteHookRequest - 296, // 61: vtctldata.ExecuteHookResponse.hook_result:type_name -> tabletmanagerdata.ExecuteHookResponse - 283, // 62: vtctldata.ExecuteMultiFetchAsDBARequest.tablet_alias:type_name -> topodata.TabletAlias - 294, // 63: vtctldata.ExecuteMultiFetchAsDBAResponse.results:type_name -> query.QueryResult - 253, // 64: vtctldata.FindAllShardsInKeyspaceResponse.shards:type_name -> vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry - 254, // 65: vtctldata.ForceCutOverSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.ForceCutOverSchemaMigrationResponse.RowsAffectedByShardEntry - 297, // 66: vtctldata.GetBackupsResponse.backups:type_name -> mysqlctl.BackupInfo - 286, // 67: vtctldata.GetCellInfoResponse.cell_info:type_name -> topodata.CellInfo - 255, // 68: vtctldata.GetCellsAliasesResponse.aliases:type_name -> vtctldata.GetCellsAliasesResponse.AliasesEntry - 283, // 69: vtctldata.GetFullStatusRequest.tablet_alias:type_name -> topodata.TabletAlias - 298, // 70: vtctldata.GetFullStatusResponse.status:type_name -> replicationdata.FullStatus - 8, // 71: vtctldata.GetKeyspacesResponse.keyspaces:type_name -> vtctldata.Keyspace - 8, // 72: vtctldata.GetKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace - 283, // 73: vtctldata.GetPermissionsRequest.tablet_alias:type_name -> topodata.TabletAlias - 299, // 74: vtctldata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions - 287, // 75: vtctldata.GetRoutingRulesResponse.routing_rules:type_name -> vschema.RoutingRules - 283, // 76: vtctldata.GetSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias - 300, // 77: vtctldata.GetSchemaResponse.schema:type_name -> tabletmanagerdata.SchemaDefinition - 3, // 78: vtctldata.GetSchemaMigrationsRequest.status:type_name -> vtctldata.SchemaMigration.Status - 284, // 79: vtctldata.GetSchemaMigrationsRequest.recent:type_name -> vttime.Duration - 1, // 80: vtctldata.GetSchemaMigrationsRequest.order:type_name -> vtctldata.QueryOrdering - 9, // 81: vtctldata.GetSchemaMigrationsResponse.migrations:type_name -> vtctldata.SchemaMigration - 256, // 82: vtctldata.GetShardReplicationResponse.shard_replication_by_cell:type_name -> vtctldata.GetShardReplicationResponse.ShardReplicationByCellEntry - 10, // 83: vtctldata.GetShardResponse.shard:type_name -> vtctldata.Shard - 288, // 84: vtctldata.GetShardRoutingRulesResponse.shard_routing_rules:type_name -> vschema.ShardRoutingRules - 257, // 85: vtctldata.GetSrvKeyspaceNamesResponse.names:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry - 259, // 86: vtctldata.GetSrvKeyspacesResponse.srv_keyspaces:type_name -> vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry - 301, // 87: vtctldata.UpdateThrottlerConfigRequest.throttled_app:type_name -> topodata.ThrottledAppRule - 302, // 88: vtctldata.GetSrvVSchemaResponse.srv_v_schema:type_name -> vschema.SrvVSchema - 260, // 89: vtctldata.GetSrvVSchemasResponse.srv_v_schemas:type_name -> vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry - 283, // 90: vtctldata.GetTabletRequest.tablet_alias:type_name -> topodata.TabletAlias - 292, // 91: vtctldata.GetTabletResponse.tablet:type_name -> topodata.Tablet - 283, // 92: vtctldata.GetTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias - 291, // 93: vtctldata.GetTabletsRequest.tablet_type:type_name -> topodata.TabletType - 292, // 94: vtctldata.GetTabletsResponse.tablets:type_name -> topodata.Tablet - 109, // 95: vtctldata.GetTopologyPathResponse.cell:type_name -> vtctldata.TopologyCell - 283, // 96: vtctldata.GetVersionRequest.tablet_alias:type_name -> topodata.TabletAlias - 290, // 97: vtctldata.GetVSchemaResponse.v_schema:type_name -> vschema.Keyspace - 11, // 98: vtctldata.GetWorkflowsResponse.workflows:type_name -> vtctldata.Workflow - 283, // 99: vtctldata.InitShardPrimaryRequest.primary_elect_tablet_alias:type_name -> topodata.TabletAlias - 284, // 100: vtctldata.InitShardPrimaryRequest.wait_replicas_timeout:type_name -> vttime.Duration - 279, // 101: vtctldata.InitShardPrimaryResponse.events:type_name -> logutil.Event - 261, // 102: vtctldata.LaunchSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.LaunchSchemaMigrationResponse.RowsAffectedByShardEntry - 290, // 103: vtctldata.LookupVindexCreateRequest.vindex:type_name -> vschema.Keyspace - 291, // 104: vtctldata.LookupVindexCreateRequest.tablet_types:type_name -> topodata.TabletType - 280, // 105: vtctldata.LookupVindexCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 7, // 106: vtctldata.MaterializeCreateRequest.settings:type_name -> vtctldata.MaterializeSettings - 291, // 107: vtctldata.MigrateCreateRequest.tablet_types:type_name -> topodata.TabletType - 280, // 108: vtctldata.MigrateCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 291, // 109: vtctldata.MoveTablesCreateRequest.tablet_types:type_name -> topodata.TabletType - 280, // 110: vtctldata.MoveTablesCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 262, // 111: vtctldata.MoveTablesCreateResponse.details:type_name -> vtctldata.MoveTablesCreateResponse.TabletInfo - 283, // 112: vtctldata.PingTabletRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 113: vtctldata.PlannedReparentShardRequest.new_primary:type_name -> topodata.TabletAlias - 283, // 114: vtctldata.PlannedReparentShardRequest.avoid_primary:type_name -> topodata.TabletAlias - 284, // 115: vtctldata.PlannedReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration - 284, // 116: vtctldata.PlannedReparentShardRequest.tolerable_replication_lag:type_name -> vttime.Duration - 283, // 117: vtctldata.PlannedReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias - 279, // 118: vtctldata.PlannedReparentShardResponse.events:type_name -> logutil.Event - 283, // 119: vtctldata.RefreshStateRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 120: vtctldata.ReloadSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias - 279, // 121: vtctldata.ReloadSchemaKeyspaceResponse.events:type_name -> logutil.Event - 279, // 122: vtctldata.ReloadSchemaShardResponse.events:type_name -> logutil.Event - 283, // 123: vtctldata.ReparentTabletRequest.tablet:type_name -> topodata.TabletAlias - 283, // 124: vtctldata.ReparentTabletResponse.primary:type_name -> topodata.TabletAlias - 291, // 125: vtctldata.ReshardCreateRequest.tablet_types:type_name -> topodata.TabletType - 280, // 126: vtctldata.ReshardCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 283, // 127: vtctldata.RestoreFromBackupRequest.tablet_alias:type_name -> topodata.TabletAlias - 282, // 128: vtctldata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time - 282, // 129: vtctldata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time - 283, // 130: vtctldata.RestoreFromBackupResponse.tablet_alias:type_name -> topodata.TabletAlias - 279, // 131: vtctldata.RestoreFromBackupResponse.event:type_name -> logutil.Event - 263, // 132: vtctldata.RetrySchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.RetrySchemaMigrationResponse.RowsAffectedByShardEntry - 283, // 133: vtctldata.RunHealthCheckRequest.tablet_alias:type_name -> topodata.TabletAlias - 281, // 134: vtctldata.SetKeyspaceDurabilityPolicyResponse.keyspace:type_name -> topodata.Keyspace - 281, // 135: vtctldata.SetKeyspaceShardingInfoResponse.keyspace:type_name -> topodata.Keyspace - 285, // 136: vtctldata.SetShardIsPrimaryServingResponse.shard:type_name -> topodata.Shard - 291, // 137: vtctldata.SetShardTabletControlRequest.tablet_type:type_name -> topodata.TabletType - 285, // 138: vtctldata.SetShardTabletControlResponse.shard:type_name -> topodata.Shard - 283, // 139: vtctldata.SetWritableRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 140: vtctldata.ShardReplicationAddRequest.tablet_alias:type_name -> topodata.TabletAlias - 303, // 141: vtctldata.ShardReplicationFixResponse.error:type_name -> topodata.ShardReplicationError - 264, // 142: vtctldata.ShardReplicationPositionsResponse.replication_statuses:type_name -> vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry - 265, // 143: vtctldata.ShardReplicationPositionsResponse.tablet_map:type_name -> vtctldata.ShardReplicationPositionsResponse.TabletMapEntry - 283, // 144: vtctldata.ShardReplicationRemoveRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 145: vtctldata.SleepTabletRequest.tablet_alias:type_name -> topodata.TabletAlias - 284, // 146: vtctldata.SleepTabletRequest.duration:type_name -> vttime.Duration - 304, // 147: vtctldata.SourceShardAddRequest.key_range:type_name -> topodata.KeyRange - 285, // 148: vtctldata.SourceShardAddResponse.shard:type_name -> topodata.Shard - 285, // 149: vtctldata.SourceShardDeleteResponse.shard:type_name -> topodata.Shard - 283, // 150: vtctldata.StartReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 151: vtctldata.StopReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias - 283, // 152: vtctldata.TabletExternallyReparentedRequest.tablet:type_name -> topodata.TabletAlias - 283, // 153: vtctldata.TabletExternallyReparentedResponse.new_primary:type_name -> topodata.TabletAlias - 283, // 154: vtctldata.TabletExternallyReparentedResponse.old_primary:type_name -> topodata.TabletAlias - 286, // 155: vtctldata.UpdateCellInfoRequest.cell_info:type_name -> topodata.CellInfo - 286, // 156: vtctldata.UpdateCellInfoResponse.cell_info:type_name -> topodata.CellInfo - 305, // 157: vtctldata.UpdateCellsAliasRequest.cells_alias:type_name -> topodata.CellsAlias - 305, // 158: vtctldata.UpdateCellsAliasResponse.cells_alias:type_name -> topodata.CellsAlias - 266, // 159: vtctldata.ValidateResponse.results_by_keyspace:type_name -> vtctldata.ValidateResponse.ResultsByKeyspaceEntry - 267, // 160: vtctldata.ValidateKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry - 268, // 161: vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry - 269, // 162: vtctldata.ValidateVersionKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry - 270, // 163: vtctldata.ValidateVSchemaResponse.results_by_shard:type_name -> vtctldata.ValidateVSchemaResponse.ResultsByShardEntry - 291, // 164: vtctldata.VDiffCreateRequest.tablet_types:type_name -> topodata.TabletType - 280, // 165: vtctldata.VDiffCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 284, // 166: vtctldata.VDiffCreateRequest.filtered_replication_wait_time:type_name -> vttime.Duration - 284, // 167: vtctldata.VDiffCreateRequest.wait_update_interval:type_name -> vttime.Duration - 284, // 168: vtctldata.VDiffCreateRequest.max_diff_duration:type_name -> vttime.Duration - 271, // 169: vtctldata.VDiffShowResponse.tablet_responses:type_name -> vtctldata.VDiffShowResponse.TabletResponsesEntry - 272, // 170: vtctldata.WorkflowDeleteResponse.details:type_name -> vtctldata.WorkflowDeleteResponse.TabletInfo - 276, // 171: vtctldata.WorkflowStatusResponse.table_copy_state:type_name -> vtctldata.WorkflowStatusResponse.TableCopyStateEntry - 277, // 172: vtctldata.WorkflowStatusResponse.shard_streams:type_name -> vtctldata.WorkflowStatusResponse.ShardStreamsEntry - 291, // 173: vtctldata.WorkflowSwitchTrafficRequest.tablet_types:type_name -> topodata.TabletType - 284, // 174: vtctldata.WorkflowSwitchTrafficRequest.max_replication_lag_allowed:type_name -> vttime.Duration - 284, // 175: vtctldata.WorkflowSwitchTrafficRequest.timeout:type_name -> vttime.Duration - 306, // 176: vtctldata.WorkflowUpdateRequest.tablet_request:type_name -> tabletmanagerdata.UpdateVReplicationWorkflowRequest - 278, // 177: vtctldata.WorkflowUpdateResponse.details:type_name -> vtctldata.WorkflowUpdateResponse.TabletInfo - 242, // 178: vtctldata.Workflow.ShardStreamsEntry.value:type_name -> vtctldata.Workflow.ShardStream - 243, // 179: vtctldata.Workflow.ShardStream.streams:type_name -> vtctldata.Workflow.Stream - 307, // 180: vtctldata.Workflow.ShardStream.tablet_controls:type_name -> topodata.Shard.TabletControl - 283, // 181: vtctldata.Workflow.Stream.tablet:type_name -> topodata.TabletAlias - 308, // 182: vtctldata.Workflow.Stream.binlog_source:type_name -> binlogdata.BinlogSource - 282, // 183: vtctldata.Workflow.Stream.transaction_timestamp:type_name -> vttime.Time - 282, // 184: vtctldata.Workflow.Stream.time_updated:type_name -> vttime.Time - 244, // 185: vtctldata.Workflow.Stream.copy_states:type_name -> vtctldata.Workflow.Stream.CopyState - 245, // 186: vtctldata.Workflow.Stream.logs:type_name -> vtctldata.Workflow.Stream.Log - 246, // 187: vtctldata.Workflow.Stream.throttler_status:type_name -> vtctldata.Workflow.Stream.ThrottlerStatus - 291, // 188: vtctldata.Workflow.Stream.tablet_types:type_name -> topodata.TabletType - 280, // 189: vtctldata.Workflow.Stream.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 282, // 190: vtctldata.Workflow.Stream.Log.created_at:type_name -> vttime.Time - 282, // 191: vtctldata.Workflow.Stream.Log.updated_at:type_name -> vttime.Time - 282, // 192: vtctldata.Workflow.Stream.ThrottlerStatus.time_throttled:type_name -> vttime.Time - 249, // 193: vtctldata.ApplyVSchemaResponse.UnknownVindexParamsEntry.value:type_name -> vtctldata.ApplyVSchemaResponse.ParamList - 10, // 194: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry.value:type_name -> vtctldata.Shard - 305, // 195: vtctldata.GetCellsAliasesResponse.AliasesEntry.value:type_name -> topodata.CellsAlias - 309, // 196: vtctldata.GetShardReplicationResponse.ShardReplicationByCellEntry.value:type_name -> topodata.ShardReplication - 258, // 197: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry.value:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NameList - 310, // 198: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry.value:type_name -> topodata.SrvKeyspace - 302, // 199: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry.value:type_name -> vschema.SrvVSchema - 283, // 200: vtctldata.MoveTablesCreateResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias - 311, // 201: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry.value:type_name -> replicationdata.Status - 292, // 202: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry.value:type_name -> topodata.Tablet - 211, // 203: vtctldata.ValidateResponse.ResultsByKeyspaceEntry.value:type_name -> vtctldata.ValidateKeyspaceResponse - 215, // 204: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 215, // 205: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 215, // 206: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 215, // 207: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse - 312, // 208: vtctldata.VDiffShowResponse.TabletResponsesEntry.value:type_name -> tabletmanagerdata.VDiffResponse - 283, // 209: vtctldata.WorkflowDeleteResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias - 283, // 210: vtctldata.WorkflowStatusResponse.ShardStreamState.tablet:type_name -> topodata.TabletAlias - 274, // 211: vtctldata.WorkflowStatusResponse.ShardStreams.streams:type_name -> vtctldata.WorkflowStatusResponse.ShardStreamState - 273, // 212: vtctldata.WorkflowStatusResponse.TableCopyStateEntry.value:type_name -> vtctldata.WorkflowStatusResponse.TableCopyState - 275, // 213: vtctldata.WorkflowStatusResponse.ShardStreamsEntry.value:type_name -> vtctldata.WorkflowStatusResponse.ShardStreams - 283, // 214: vtctldata.WorkflowUpdateResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias - 215, // [215:215] is the sub-list for method output_type - 215, // [215:215] is the sub-list for method input_type - 215, // [215:215] is the sub-list for extension type_name - 215, // [215:215] is the sub-list for extension extendee - 0, // [0:215] is the sub-list for field type_name + 285, // 3: vtctldata.MaterializeSettings.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 11, // 4: vtctldata.MaterializeSettings.workflow_options:type_name -> vtctldata.WorkflowOptions + 286, // 5: vtctldata.Keyspace.keyspace:type_name -> topodata.Keyspace + 2, // 6: vtctldata.SchemaMigration.strategy:type_name -> vtctldata.SchemaMigration.Strategy + 287, // 7: vtctldata.SchemaMigration.added_at:type_name -> vttime.Time + 287, // 8: vtctldata.SchemaMigration.requested_at:type_name -> vttime.Time + 287, // 9: vtctldata.SchemaMigration.ready_at:type_name -> vttime.Time + 287, // 10: vtctldata.SchemaMigration.started_at:type_name -> vttime.Time + 287, // 11: vtctldata.SchemaMigration.liveness_timestamp:type_name -> vttime.Time + 287, // 12: vtctldata.SchemaMigration.completed_at:type_name -> vttime.Time + 287, // 13: vtctldata.SchemaMigration.cleaned_up_at:type_name -> vttime.Time + 3, // 14: vtctldata.SchemaMigration.status:type_name -> vtctldata.SchemaMigration.Status + 288, // 15: vtctldata.SchemaMigration.tablet:type_name -> topodata.TabletAlias + 289, // 16: vtctldata.SchemaMigration.artifact_retention:type_name -> vttime.Duration + 287, // 17: vtctldata.SchemaMigration.last_throttled_at:type_name -> vttime.Time + 287, // 18: vtctldata.SchemaMigration.cancelled_at:type_name -> vttime.Time + 287, // 19: vtctldata.SchemaMigration.reviewed_at:type_name -> vttime.Time + 287, // 20: vtctldata.SchemaMigration.ready_to_complete_at:type_name -> vttime.Time + 290, // 21: vtctldata.Shard.shard:type_name -> topodata.Shard + 246, // 22: vtctldata.Workflow.source:type_name -> vtctldata.Workflow.ReplicationLocation + 246, // 23: vtctldata.Workflow.target:type_name -> vtctldata.Workflow.ReplicationLocation + 245, // 24: vtctldata.Workflow.shard_streams:type_name -> vtctldata.Workflow.ShardStreamsEntry + 11, // 25: vtctldata.Workflow.options:type_name -> vtctldata.WorkflowOptions + 291, // 26: vtctldata.AddCellInfoRequest.cell_info:type_name -> topodata.CellInfo + 292, // 27: vtctldata.ApplyKeyspaceRoutingRulesRequest.keyspace_routing_rules:type_name -> vschema.KeyspaceRoutingRules + 293, // 28: vtctldata.ApplyRoutingRulesRequest.routing_rules:type_name -> vschema.RoutingRules + 294, // 29: vtctldata.ApplyShardRoutingRulesRequest.shard_routing_rules:type_name -> vschema.ShardRoutingRules + 289, // 30: vtctldata.ApplySchemaRequest.wait_replicas_timeout:type_name -> vttime.Duration + 295, // 31: vtctldata.ApplySchemaRequest.caller_id:type_name -> vtrpc.CallerID + 252, // 32: vtctldata.ApplySchemaResponse.rows_affected_by_shard:type_name -> vtctldata.ApplySchemaResponse.RowsAffectedByShardEntry + 296, // 33: vtctldata.ApplyVSchemaRequest.v_schema:type_name -> vschema.Keyspace + 296, // 34: vtctldata.ApplyVSchemaResponse.v_schema:type_name -> vschema.Keyspace + 253, // 35: vtctldata.ApplyVSchemaResponse.unknown_vindex_params:type_name -> vtctldata.ApplyVSchemaResponse.UnknownVindexParamsEntry + 288, // 36: vtctldata.BackupRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 37: vtctldata.BackupResponse.tablet_alias:type_name -> topodata.TabletAlias + 284, // 38: vtctldata.BackupResponse.event:type_name -> logutil.Event + 255, // 39: vtctldata.CancelSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.CancelSchemaMigrationResponse.RowsAffectedByShardEntry + 288, // 40: vtctldata.ChangeTabletTypeRequest.tablet_alias:type_name -> topodata.TabletAlias + 297, // 41: vtctldata.ChangeTabletTypeRequest.db_type:type_name -> topodata.TabletType + 298, // 42: vtctldata.ChangeTabletTypeResponse.before_tablet:type_name -> topodata.Tablet + 298, // 43: vtctldata.ChangeTabletTypeResponse.after_tablet:type_name -> topodata.Tablet + 256, // 44: vtctldata.CleanupSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.CleanupSchemaMigrationResponse.RowsAffectedByShardEntry + 257, // 45: vtctldata.CompleteSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.CompleteSchemaMigrationResponse.RowsAffectedByShardEntry + 299, // 46: vtctldata.CreateKeyspaceRequest.type:type_name -> topodata.KeyspaceType + 287, // 47: vtctldata.CreateKeyspaceRequest.snapshot_time:type_name -> vttime.Time + 8, // 48: vtctldata.CreateKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace + 8, // 49: vtctldata.CreateShardResponse.keyspace:type_name -> vtctldata.Keyspace + 10, // 50: vtctldata.CreateShardResponse.shard:type_name -> vtctldata.Shard + 10, // 51: vtctldata.DeleteShardsRequest.shards:type_name -> vtctldata.Shard + 288, // 52: vtctldata.DeleteTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias + 288, // 53: vtctldata.EmergencyReparentShardRequest.new_primary:type_name -> topodata.TabletAlias + 288, // 54: vtctldata.EmergencyReparentShardRequest.ignore_replicas:type_name -> topodata.TabletAlias + 289, // 55: vtctldata.EmergencyReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration + 288, // 56: vtctldata.EmergencyReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias + 284, // 57: vtctldata.EmergencyReparentShardResponse.events:type_name -> logutil.Event + 288, // 58: vtctldata.ExecuteFetchAsAppRequest.tablet_alias:type_name -> topodata.TabletAlias + 300, // 59: vtctldata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult + 288, // 60: vtctldata.ExecuteFetchAsDBARequest.tablet_alias:type_name -> topodata.TabletAlias + 300, // 61: vtctldata.ExecuteFetchAsDBAResponse.result:type_name -> query.QueryResult + 288, // 62: vtctldata.ExecuteHookRequest.tablet_alias:type_name -> topodata.TabletAlias + 301, // 63: vtctldata.ExecuteHookRequest.tablet_hook_request:type_name -> tabletmanagerdata.ExecuteHookRequest + 302, // 64: vtctldata.ExecuteHookResponse.hook_result:type_name -> tabletmanagerdata.ExecuteHookResponse + 288, // 65: vtctldata.ExecuteMultiFetchAsDBARequest.tablet_alias:type_name -> topodata.TabletAlias + 300, // 66: vtctldata.ExecuteMultiFetchAsDBAResponse.results:type_name -> query.QueryResult + 258, // 67: vtctldata.FindAllShardsInKeyspaceResponse.shards:type_name -> vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry + 259, // 68: vtctldata.ForceCutOverSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.ForceCutOverSchemaMigrationResponse.RowsAffectedByShardEntry + 303, // 69: vtctldata.GetBackupsResponse.backups:type_name -> mysqlctl.BackupInfo + 291, // 70: vtctldata.GetCellInfoResponse.cell_info:type_name -> topodata.CellInfo + 260, // 71: vtctldata.GetCellsAliasesResponse.aliases:type_name -> vtctldata.GetCellsAliasesResponse.AliasesEntry + 288, // 72: vtctldata.GetFullStatusRequest.tablet_alias:type_name -> topodata.TabletAlias + 304, // 73: vtctldata.GetFullStatusResponse.status:type_name -> replicationdata.FullStatus + 8, // 74: vtctldata.GetKeyspacesResponse.keyspaces:type_name -> vtctldata.Keyspace + 8, // 75: vtctldata.GetKeyspaceResponse.keyspace:type_name -> vtctldata.Keyspace + 288, // 76: vtctldata.GetPermissionsRequest.tablet_alias:type_name -> topodata.TabletAlias + 305, // 77: vtctldata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions + 292, // 78: vtctldata.GetKeyspaceRoutingRulesResponse.keyspace_routing_rules:type_name -> vschema.KeyspaceRoutingRules + 293, // 79: vtctldata.GetRoutingRulesResponse.routing_rules:type_name -> vschema.RoutingRules + 288, // 80: vtctldata.GetSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias + 306, // 81: vtctldata.GetSchemaResponse.schema:type_name -> tabletmanagerdata.SchemaDefinition + 3, // 82: vtctldata.GetSchemaMigrationsRequest.status:type_name -> vtctldata.SchemaMigration.Status + 289, // 83: vtctldata.GetSchemaMigrationsRequest.recent:type_name -> vttime.Duration + 1, // 84: vtctldata.GetSchemaMigrationsRequest.order:type_name -> vtctldata.QueryOrdering + 9, // 85: vtctldata.GetSchemaMigrationsResponse.migrations:type_name -> vtctldata.SchemaMigration + 261, // 86: vtctldata.GetShardReplicationResponse.shard_replication_by_cell:type_name -> vtctldata.GetShardReplicationResponse.ShardReplicationByCellEntry + 10, // 87: vtctldata.GetShardResponse.shard:type_name -> vtctldata.Shard + 294, // 88: vtctldata.GetShardRoutingRulesResponse.shard_routing_rules:type_name -> vschema.ShardRoutingRules + 262, // 89: vtctldata.GetSrvKeyspaceNamesResponse.names:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry + 264, // 90: vtctldata.GetSrvKeyspacesResponse.srv_keyspaces:type_name -> vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry + 307, // 91: vtctldata.UpdateThrottlerConfigRequest.throttled_app:type_name -> topodata.ThrottledAppRule + 308, // 92: vtctldata.GetSrvVSchemaResponse.srv_v_schema:type_name -> vschema.SrvVSchema + 265, // 93: vtctldata.GetSrvVSchemasResponse.srv_v_schemas:type_name -> vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry + 288, // 94: vtctldata.GetTabletRequest.tablet_alias:type_name -> topodata.TabletAlias + 298, // 95: vtctldata.GetTabletResponse.tablet:type_name -> topodata.Tablet + 288, // 96: vtctldata.GetTabletsRequest.tablet_aliases:type_name -> topodata.TabletAlias + 297, // 97: vtctldata.GetTabletsRequest.tablet_type:type_name -> topodata.TabletType + 298, // 98: vtctldata.GetTabletsResponse.tablets:type_name -> topodata.Tablet + 114, // 99: vtctldata.GetTopologyPathResponse.cell:type_name -> vtctldata.TopologyCell + 288, // 100: vtctldata.GetVersionRequest.tablet_alias:type_name -> topodata.TabletAlias + 296, // 101: vtctldata.GetVSchemaResponse.v_schema:type_name -> vschema.Keyspace + 12, // 102: vtctldata.GetWorkflowsResponse.workflows:type_name -> vtctldata.Workflow + 288, // 103: vtctldata.InitShardPrimaryRequest.primary_elect_tablet_alias:type_name -> topodata.TabletAlias + 289, // 104: vtctldata.InitShardPrimaryRequest.wait_replicas_timeout:type_name -> vttime.Duration + 284, // 105: vtctldata.InitShardPrimaryResponse.events:type_name -> logutil.Event + 266, // 106: vtctldata.LaunchSchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.LaunchSchemaMigrationResponse.RowsAffectedByShardEntry + 296, // 107: vtctldata.LookupVindexCreateRequest.vindex:type_name -> vschema.Keyspace + 297, // 108: vtctldata.LookupVindexCreateRequest.tablet_types:type_name -> topodata.TabletType + 285, // 109: vtctldata.LookupVindexCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 7, // 110: vtctldata.MaterializeCreateRequest.settings:type_name -> vtctldata.MaterializeSettings + 297, // 111: vtctldata.MigrateCreateRequest.tablet_types:type_name -> topodata.TabletType + 285, // 112: vtctldata.MigrateCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 297, // 113: vtctldata.MoveTablesCreateRequest.tablet_types:type_name -> topodata.TabletType + 285, // 114: vtctldata.MoveTablesCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 11, // 115: vtctldata.MoveTablesCreateRequest.workflow_options:type_name -> vtctldata.WorkflowOptions + 267, // 116: vtctldata.MoveTablesCreateResponse.details:type_name -> vtctldata.MoveTablesCreateResponse.TabletInfo + 288, // 117: vtctldata.PingTabletRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 118: vtctldata.PlannedReparentShardRequest.new_primary:type_name -> topodata.TabletAlias + 288, // 119: vtctldata.PlannedReparentShardRequest.avoid_primary:type_name -> topodata.TabletAlias + 289, // 120: vtctldata.PlannedReparentShardRequest.wait_replicas_timeout:type_name -> vttime.Duration + 289, // 121: vtctldata.PlannedReparentShardRequest.tolerable_replication_lag:type_name -> vttime.Duration + 288, // 122: vtctldata.PlannedReparentShardResponse.promoted_primary:type_name -> topodata.TabletAlias + 284, // 123: vtctldata.PlannedReparentShardResponse.events:type_name -> logutil.Event + 288, // 124: vtctldata.RefreshStateRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 125: vtctldata.ReloadSchemaRequest.tablet_alias:type_name -> topodata.TabletAlias + 284, // 126: vtctldata.ReloadSchemaKeyspaceResponse.events:type_name -> logutil.Event + 284, // 127: vtctldata.ReloadSchemaShardResponse.events:type_name -> logutil.Event + 288, // 128: vtctldata.ReparentTabletRequest.tablet:type_name -> topodata.TabletAlias + 288, // 129: vtctldata.ReparentTabletResponse.primary:type_name -> topodata.TabletAlias + 297, // 130: vtctldata.ReshardCreateRequest.tablet_types:type_name -> topodata.TabletType + 285, // 131: vtctldata.ReshardCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 288, // 132: vtctldata.RestoreFromBackupRequest.tablet_alias:type_name -> topodata.TabletAlias + 287, // 133: vtctldata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time + 287, // 134: vtctldata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time + 288, // 135: vtctldata.RestoreFromBackupResponse.tablet_alias:type_name -> topodata.TabletAlias + 284, // 136: vtctldata.RestoreFromBackupResponse.event:type_name -> logutil.Event + 268, // 137: vtctldata.RetrySchemaMigrationResponse.rows_affected_by_shard:type_name -> vtctldata.RetrySchemaMigrationResponse.RowsAffectedByShardEntry + 288, // 138: vtctldata.RunHealthCheckRequest.tablet_alias:type_name -> topodata.TabletAlias + 286, // 139: vtctldata.SetKeyspaceDurabilityPolicyResponse.keyspace:type_name -> topodata.Keyspace + 286, // 140: vtctldata.SetKeyspaceShardingInfoResponse.keyspace:type_name -> topodata.Keyspace + 290, // 141: vtctldata.SetShardIsPrimaryServingResponse.shard:type_name -> topodata.Shard + 297, // 142: vtctldata.SetShardTabletControlRequest.tablet_type:type_name -> topodata.TabletType + 290, // 143: vtctldata.SetShardTabletControlResponse.shard:type_name -> topodata.Shard + 288, // 144: vtctldata.SetWritableRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 145: vtctldata.ShardReplicationAddRequest.tablet_alias:type_name -> topodata.TabletAlias + 309, // 146: vtctldata.ShardReplicationFixResponse.error:type_name -> topodata.ShardReplicationError + 269, // 147: vtctldata.ShardReplicationPositionsResponse.replication_statuses:type_name -> vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry + 270, // 148: vtctldata.ShardReplicationPositionsResponse.tablet_map:type_name -> vtctldata.ShardReplicationPositionsResponse.TabletMapEntry + 288, // 149: vtctldata.ShardReplicationRemoveRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 150: vtctldata.SleepTabletRequest.tablet_alias:type_name -> topodata.TabletAlias + 289, // 151: vtctldata.SleepTabletRequest.duration:type_name -> vttime.Duration + 310, // 152: vtctldata.SourceShardAddRequest.key_range:type_name -> topodata.KeyRange + 290, // 153: vtctldata.SourceShardAddResponse.shard:type_name -> topodata.Shard + 290, // 154: vtctldata.SourceShardDeleteResponse.shard:type_name -> topodata.Shard + 288, // 155: vtctldata.StartReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 156: vtctldata.StopReplicationRequest.tablet_alias:type_name -> topodata.TabletAlias + 288, // 157: vtctldata.TabletExternallyReparentedRequest.tablet:type_name -> topodata.TabletAlias + 288, // 158: vtctldata.TabletExternallyReparentedResponse.new_primary:type_name -> topodata.TabletAlias + 288, // 159: vtctldata.TabletExternallyReparentedResponse.old_primary:type_name -> topodata.TabletAlias + 291, // 160: vtctldata.UpdateCellInfoRequest.cell_info:type_name -> topodata.CellInfo + 291, // 161: vtctldata.UpdateCellInfoResponse.cell_info:type_name -> topodata.CellInfo + 311, // 162: vtctldata.UpdateCellsAliasRequest.cells_alias:type_name -> topodata.CellsAlias + 311, // 163: vtctldata.UpdateCellsAliasResponse.cells_alias:type_name -> topodata.CellsAlias + 271, // 164: vtctldata.ValidateResponse.results_by_keyspace:type_name -> vtctldata.ValidateResponse.ResultsByKeyspaceEntry + 272, // 165: vtctldata.ValidateKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry + 273, // 166: vtctldata.ValidateSchemaKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry + 274, // 167: vtctldata.ValidateVersionKeyspaceResponse.results_by_shard:type_name -> vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry + 275, // 168: vtctldata.ValidateVSchemaResponse.results_by_shard:type_name -> vtctldata.ValidateVSchemaResponse.ResultsByShardEntry + 297, // 169: vtctldata.VDiffCreateRequest.tablet_types:type_name -> topodata.TabletType + 285, // 170: vtctldata.VDiffCreateRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 289, // 171: vtctldata.VDiffCreateRequest.filtered_replication_wait_time:type_name -> vttime.Duration + 289, // 172: vtctldata.VDiffCreateRequest.wait_update_interval:type_name -> vttime.Duration + 289, // 173: vtctldata.VDiffCreateRequest.max_diff_duration:type_name -> vttime.Duration + 276, // 174: vtctldata.VDiffShowResponse.tablet_responses:type_name -> vtctldata.VDiffShowResponse.TabletResponsesEntry + 277, // 175: vtctldata.WorkflowDeleteResponse.details:type_name -> vtctldata.WorkflowDeleteResponse.TabletInfo + 281, // 176: vtctldata.WorkflowStatusResponse.table_copy_state:type_name -> vtctldata.WorkflowStatusResponse.TableCopyStateEntry + 282, // 177: vtctldata.WorkflowStatusResponse.shard_streams:type_name -> vtctldata.WorkflowStatusResponse.ShardStreamsEntry + 297, // 178: vtctldata.WorkflowSwitchTrafficRequest.tablet_types:type_name -> topodata.TabletType + 289, // 179: vtctldata.WorkflowSwitchTrafficRequest.max_replication_lag_allowed:type_name -> vttime.Duration + 289, // 180: vtctldata.WorkflowSwitchTrafficRequest.timeout:type_name -> vttime.Duration + 312, // 181: vtctldata.WorkflowUpdateRequest.tablet_request:type_name -> tabletmanagerdata.UpdateVReplicationWorkflowRequest + 283, // 182: vtctldata.WorkflowUpdateResponse.details:type_name -> vtctldata.WorkflowUpdateResponse.TabletInfo + 247, // 183: vtctldata.Workflow.ShardStreamsEntry.value:type_name -> vtctldata.Workflow.ShardStream + 248, // 184: vtctldata.Workflow.ShardStream.streams:type_name -> vtctldata.Workflow.Stream + 313, // 185: vtctldata.Workflow.ShardStream.tablet_controls:type_name -> topodata.Shard.TabletControl + 288, // 186: vtctldata.Workflow.Stream.tablet:type_name -> topodata.TabletAlias + 314, // 187: vtctldata.Workflow.Stream.binlog_source:type_name -> binlogdata.BinlogSource + 287, // 188: vtctldata.Workflow.Stream.transaction_timestamp:type_name -> vttime.Time + 287, // 189: vtctldata.Workflow.Stream.time_updated:type_name -> vttime.Time + 249, // 190: vtctldata.Workflow.Stream.copy_states:type_name -> vtctldata.Workflow.Stream.CopyState + 250, // 191: vtctldata.Workflow.Stream.logs:type_name -> vtctldata.Workflow.Stream.Log + 251, // 192: vtctldata.Workflow.Stream.throttler_status:type_name -> vtctldata.Workflow.Stream.ThrottlerStatus + 297, // 193: vtctldata.Workflow.Stream.tablet_types:type_name -> topodata.TabletType + 285, // 194: vtctldata.Workflow.Stream.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 287, // 195: vtctldata.Workflow.Stream.Log.created_at:type_name -> vttime.Time + 287, // 196: vtctldata.Workflow.Stream.Log.updated_at:type_name -> vttime.Time + 287, // 197: vtctldata.Workflow.Stream.ThrottlerStatus.time_throttled:type_name -> vttime.Time + 254, // 198: vtctldata.ApplyVSchemaResponse.UnknownVindexParamsEntry.value:type_name -> vtctldata.ApplyVSchemaResponse.ParamList + 10, // 199: vtctldata.FindAllShardsInKeyspaceResponse.ShardsEntry.value:type_name -> vtctldata.Shard + 311, // 200: vtctldata.GetCellsAliasesResponse.AliasesEntry.value:type_name -> topodata.CellsAlias + 315, // 201: vtctldata.GetShardReplicationResponse.ShardReplicationByCellEntry.value:type_name -> topodata.ShardReplication + 263, // 202: vtctldata.GetSrvKeyspaceNamesResponse.NamesEntry.value:type_name -> vtctldata.GetSrvKeyspaceNamesResponse.NameList + 316, // 203: vtctldata.GetSrvKeyspacesResponse.SrvKeyspacesEntry.value:type_name -> topodata.SrvKeyspace + 308, // 204: vtctldata.GetSrvVSchemasResponse.SrvVSchemasEntry.value:type_name -> vschema.SrvVSchema + 288, // 205: vtctldata.MoveTablesCreateResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias + 317, // 206: vtctldata.ShardReplicationPositionsResponse.ReplicationStatusesEntry.value:type_name -> replicationdata.Status + 298, // 207: vtctldata.ShardReplicationPositionsResponse.TabletMapEntry.value:type_name -> topodata.Tablet + 216, // 208: vtctldata.ValidateResponse.ResultsByKeyspaceEntry.value:type_name -> vtctldata.ValidateKeyspaceResponse + 220, // 209: vtctldata.ValidateKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 220, // 210: vtctldata.ValidateSchemaKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 220, // 211: vtctldata.ValidateVersionKeyspaceResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 220, // 212: vtctldata.ValidateVSchemaResponse.ResultsByShardEntry.value:type_name -> vtctldata.ValidateShardResponse + 318, // 213: vtctldata.VDiffShowResponse.TabletResponsesEntry.value:type_name -> tabletmanagerdata.VDiffResponse + 288, // 214: vtctldata.WorkflowDeleteResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias + 288, // 215: vtctldata.WorkflowStatusResponse.ShardStreamState.tablet:type_name -> topodata.TabletAlias + 279, // 216: vtctldata.WorkflowStatusResponse.ShardStreams.streams:type_name -> vtctldata.WorkflowStatusResponse.ShardStreamState + 278, // 217: vtctldata.WorkflowStatusResponse.TableCopyStateEntry.value:type_name -> vtctldata.WorkflowStatusResponse.TableCopyState + 280, // 218: vtctldata.WorkflowStatusResponse.ShardStreamsEntry.value:type_name -> vtctldata.WorkflowStatusResponse.ShardStreams + 288, // 219: vtctldata.WorkflowUpdateResponse.TabletInfo.tablet:type_name -> topodata.TabletAlias + 220, // [220:220] is the sub-list for method output_type + 220, // [220:220] is the sub-list for method input_type + 220, // [220:220] is the sub-list for extension type_name + 220, // [220:220] is the sub-list for extension extendee + 0, // [0:220] is the sub-list for field type_name } func init() { file_vtctldata_proto_init() } @@ -18732,8 +19059,44 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MaterializeSettings); i { + file_vtctldata_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MaterializeSettings); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Keyspace); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SchemaMigration); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Shard); i { case 0: return &v.state case 1: @@ -18744,8 +19107,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Keyspace); i { + file_vtctldata_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkflowOptions); i { case 0: return &v.state case 1: @@ -18756,8 +19119,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SchemaMigration); i { + file_vtctldata_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Workflow); i { case 0: return &v.state case 1: @@ -18768,8 +19131,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Shard); i { + file_vtctldata_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AddCellInfoRequest); i { case 0: return &v.state case 1: @@ -18780,8 +19143,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Workflow); i { + file_vtctldata_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AddCellInfoResponse); i { case 0: return &v.state case 1: @@ -18792,8 +19155,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AddCellInfoRequest); i { + file_vtctldata_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AddCellsAliasRequest); i { case 0: return &v.state case 1: @@ -18804,8 +19167,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AddCellInfoResponse); i { + file_vtctldata_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AddCellsAliasResponse); i { case 0: return &v.state case 1: @@ -18816,8 +19179,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AddCellsAliasRequest); i { + file_vtctldata_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ApplyKeyspaceRoutingRulesRequest); i { case 0: return &v.state case 1: @@ -18828,8 +19191,8 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AddCellsAliasResponse); i { + file_vtctldata_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ApplyKeyspaceRoutingRulesResponse); i { case 0: return &v.state case 1: @@ -18840,7 +19203,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyRoutingRulesRequest); i { case 0: return &v.state @@ -18852,7 +19215,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyRoutingRulesResponse); i { case 0: return &v.state @@ -18864,7 +19227,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyShardRoutingRulesRequest); i { case 0: return &v.state @@ -18876,7 +19239,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyShardRoutingRulesResponse); i { case 0: return &v.state @@ -18888,7 +19251,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplySchemaRequest); i { case 0: return &v.state @@ -18900,7 +19263,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplySchemaResponse); i { case 0: return &v.state @@ -18912,7 +19275,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyVSchemaRequest); i { case 0: return &v.state @@ -18924,7 +19287,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyVSchemaResponse); i { case 0: return &v.state @@ -18936,7 +19299,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupRequest); i { case 0: return &v.state @@ -18948,7 +19311,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupResponse); i { case 0: return &v.state @@ -18960,7 +19323,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BackupShardRequest); i { case 0: return &v.state @@ -18972,7 +19335,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CancelSchemaMigrationRequest); i { case 0: return &v.state @@ -18984,7 +19347,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CancelSchemaMigrationResponse); i { case 0: return &v.state @@ -18996,7 +19359,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ChangeTabletTypeRequest); i { case 0: return &v.state @@ -19008,7 +19371,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ChangeTabletTypeResponse); i { case 0: return &v.state @@ -19020,7 +19383,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CleanupSchemaMigrationRequest); i { case 0: return &v.state @@ -19032,7 +19395,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CleanupSchemaMigrationResponse); i { case 0: return &v.state @@ -19044,7 +19407,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CompleteSchemaMigrationRequest); i { case 0: return &v.state @@ -19056,7 +19419,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CompleteSchemaMigrationResponse); i { case 0: return &v.state @@ -19068,7 +19431,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateKeyspaceRequest); i { case 0: return &v.state @@ -19080,7 +19443,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateKeyspaceResponse); i { case 0: return &v.state @@ -19092,7 +19455,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateShardRequest); i { case 0: return &v.state @@ -19104,7 +19467,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CreateShardResponse); i { case 0: return &v.state @@ -19116,7 +19479,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteCellInfoRequest); i { case 0: return &v.state @@ -19128,7 +19491,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteCellInfoResponse); i { case 0: return &v.state @@ -19140,7 +19503,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteCellsAliasRequest); i { case 0: return &v.state @@ -19152,7 +19515,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteCellsAliasResponse); i { case 0: return &v.state @@ -19164,7 +19527,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteKeyspaceRequest); i { case 0: return &v.state @@ -19176,7 +19539,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteKeyspaceResponse); i { case 0: return &v.state @@ -19188,7 +19551,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteShardsRequest); i { case 0: return &v.state @@ -19200,7 +19563,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteShardsResponse); i { case 0: return &v.state @@ -19212,7 +19575,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteSrvVSchemaRequest); i { case 0: return &v.state @@ -19224,7 +19587,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteSrvVSchemaResponse); i { case 0: return &v.state @@ -19236,7 +19599,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteTabletsRequest); i { case 0: return &v.state @@ -19248,7 +19611,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DeleteTabletsResponse); i { case 0: return &v.state @@ -19260,7 +19623,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*EmergencyReparentShardRequest); i { case 0: return &v.state @@ -19272,7 +19635,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*EmergencyReparentShardResponse); i { case 0: return &v.state @@ -19284,7 +19647,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteFetchAsAppRequest); i { case 0: return &v.state @@ -19296,7 +19659,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteFetchAsAppResponse); i { case 0: return &v.state @@ -19308,7 +19671,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteFetchAsDBARequest); i { case 0: return &v.state @@ -19320,7 +19683,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteFetchAsDBAResponse); i { case 0: return &v.state @@ -19332,7 +19695,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteHookRequest); i { case 0: return &v.state @@ -19344,7 +19707,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteHookResponse); i { case 0: return &v.state @@ -19356,7 +19719,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteMultiFetchAsDBARequest); i { case 0: return &v.state @@ -19368,7 +19731,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecuteMultiFetchAsDBAResponse); i { case 0: return &v.state @@ -19380,7 +19743,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*FindAllShardsInKeyspaceRequest); i { case 0: return &v.state @@ -19392,7 +19755,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*FindAllShardsInKeyspaceResponse); i { case 0: return &v.state @@ -19404,7 +19767,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ForceCutOverSchemaMigrationRequest); i { case 0: return &v.state @@ -19416,7 +19779,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ForceCutOverSchemaMigrationResponse); i { case 0: return &v.state @@ -19428,7 +19791,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetBackupsRequest); i { case 0: return &v.state @@ -19440,7 +19803,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetBackupsResponse); i { case 0: return &v.state @@ -19452,7 +19815,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetCellInfoRequest); i { case 0: return &v.state @@ -19464,7 +19827,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetCellInfoResponse); i { case 0: return &v.state @@ -19476,7 +19839,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetCellInfoNamesRequest); i { case 0: return &v.state @@ -19488,7 +19851,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetCellInfoNamesResponse); i { case 0: return &v.state @@ -19500,7 +19863,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetCellsAliasesRequest); i { case 0: return &v.state @@ -19512,7 +19875,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetCellsAliasesResponse); i { case 0: return &v.state @@ -19524,7 +19887,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetFullStatusRequest); i { case 0: return &v.state @@ -19536,7 +19899,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetFullStatusResponse); i { case 0: return &v.state @@ -19548,7 +19911,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetKeyspacesRequest); i { case 0: return &v.state @@ -19560,7 +19923,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetKeyspacesResponse); i { case 0: return &v.state @@ -19572,7 +19935,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetKeyspaceRequest); i { case 0: return &v.state @@ -19584,7 +19947,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetKeyspaceResponse); i { case 0: return &v.state @@ -19596,7 +19959,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetPermissionsRequest); i { case 0: return &v.state @@ -19608,7 +19971,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetPermissionsResponse); i { case 0: return &v.state @@ -19620,7 +19983,31 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetKeyspaceRoutingRulesRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetKeyspaceRoutingRulesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_vtctldata_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetRoutingRulesRequest); i { case 0: return &v.state @@ -19632,7 +20019,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetRoutingRulesResponse); i { case 0: return &v.state @@ -19644,7 +20031,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSchemaRequest); i { case 0: return &v.state @@ -19656,7 +20043,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSchemaResponse); i { case 0: return &v.state @@ -19668,7 +20055,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSchemaMigrationsRequest); i { case 0: return &v.state @@ -19680,7 +20067,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSchemaMigrationsResponse); i { case 0: return &v.state @@ -19692,7 +20079,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetShardReplicationRequest); i { case 0: return &v.state @@ -19704,7 +20091,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetShardReplicationResponse); i { case 0: return &v.state @@ -19716,7 +20103,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetShardRequest); i { case 0: return &v.state @@ -19728,7 +20115,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetShardResponse); i { case 0: return &v.state @@ -19740,7 +20127,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetShardRoutingRulesRequest); i { case 0: return &v.state @@ -19752,7 +20139,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetShardRoutingRulesResponse); i { case 0: return &v.state @@ -19764,7 +20151,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvKeyspaceNamesRequest); i { case 0: return &v.state @@ -19776,7 +20163,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvKeyspaceNamesResponse); i { case 0: return &v.state @@ -19788,7 +20175,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvKeyspacesRequest); i { case 0: return &v.state @@ -19800,7 +20187,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvKeyspacesResponse); i { case 0: return &v.state @@ -19812,7 +20199,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateThrottlerConfigRequest); i { case 0: return &v.state @@ -19824,7 +20211,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateThrottlerConfigResponse); i { case 0: return &v.state @@ -19836,7 +20223,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvVSchemaRequest); i { case 0: return &v.state @@ -19848,7 +20235,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvVSchemaResponse); i { case 0: return &v.state @@ -19860,7 +20247,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvVSchemasRequest); i { case 0: return &v.state @@ -19872,7 +20259,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvVSchemasResponse); i { case 0: return &v.state @@ -19884,7 +20271,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTabletRequest); i { case 0: return &v.state @@ -19896,7 +20283,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTabletResponse); i { case 0: return &v.state @@ -19908,7 +20295,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTabletsRequest); i { case 0: return &v.state @@ -19920,7 +20307,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTabletsResponse); i { case 0: return &v.state @@ -19932,7 +20319,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTopologyPathRequest); i { case 0: return &v.state @@ -19944,7 +20331,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTopologyPathResponse); i { case 0: return &v.state @@ -19956,7 +20343,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TopologyCell); i { case 0: return &v.state @@ -19968,7 +20355,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetVSchemaRequest); i { case 0: return &v.state @@ -19980,7 +20367,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetVersionRequest); i { case 0: return &v.state @@ -19992,7 +20379,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetVersionResponse); i { case 0: return &v.state @@ -20004,7 +20391,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetVSchemaResponse); i { case 0: return &v.state @@ -20016,7 +20403,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetWorkflowsRequest); i { case 0: return &v.state @@ -20028,7 +20415,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetWorkflowsResponse); i { case 0: return &v.state @@ -20040,7 +20427,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*InitShardPrimaryRequest); i { case 0: return &v.state @@ -20052,7 +20439,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*InitShardPrimaryResponse); i { case 0: return &v.state @@ -20064,7 +20451,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LaunchSchemaMigrationRequest); i { case 0: return &v.state @@ -20076,7 +20463,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LaunchSchemaMigrationResponse); i { case 0: return &v.state @@ -20088,7 +20475,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LookupVindexCreateRequest); i { case 0: return &v.state @@ -20100,7 +20487,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LookupVindexCreateResponse); i { case 0: return &v.state @@ -20112,7 +20499,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LookupVindexExternalizeRequest); i { case 0: return &v.state @@ -20124,7 +20511,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LookupVindexExternalizeResponse); i { case 0: return &v.state @@ -20136,7 +20523,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[125].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MaterializeCreateRequest); i { case 0: return &v.state @@ -20148,7 +20535,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MaterializeCreateResponse); i { case 0: return &v.state @@ -20160,7 +20547,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[127].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MigrateCreateRequest); i { case 0: return &v.state @@ -20172,7 +20559,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[128].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MigrateCompleteRequest); i { case 0: return &v.state @@ -20184,7 +20571,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[129].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MigrateCompleteResponse); i { case 0: return &v.state @@ -20196,7 +20583,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[125].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[130].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountRegisterRequest); i { case 0: return &v.state @@ -20208,7 +20595,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[131].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountRegisterResponse); i { case 0: return &v.state @@ -20220,7 +20607,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[127].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[132].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountUnregisterRequest); i { case 0: return &v.state @@ -20232,7 +20619,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[128].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[133].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountUnregisterResponse); i { case 0: return &v.state @@ -20244,7 +20631,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[129].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[134].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountShowRequest); i { case 0: return &v.state @@ -20256,7 +20643,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[130].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[135].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountShowResponse); i { case 0: return &v.state @@ -20268,7 +20655,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[131].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[136].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountListRequest); i { case 0: return &v.state @@ -20280,7 +20667,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[132].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[137].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MountListResponse); i { case 0: return &v.state @@ -20292,7 +20679,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[133].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[138].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MoveTablesCreateRequest); i { case 0: return &v.state @@ -20304,7 +20691,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[134].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[139].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MoveTablesCreateResponse); i { case 0: return &v.state @@ -20316,7 +20703,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[135].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[140].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MoveTablesCompleteRequest); i { case 0: return &v.state @@ -20328,7 +20715,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[136].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[141].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MoveTablesCompleteResponse); i { case 0: return &v.state @@ -20340,7 +20727,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[137].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[142].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PingTabletRequest); i { case 0: return &v.state @@ -20352,7 +20739,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[138].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[143].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PingTabletResponse); i { case 0: return &v.state @@ -20364,7 +20751,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[139].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[144].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PlannedReparentShardRequest); i { case 0: return &v.state @@ -20376,7 +20763,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[140].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[145].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*PlannedReparentShardResponse); i { case 0: return &v.state @@ -20388,7 +20775,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[141].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[146].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RebuildKeyspaceGraphRequest); i { case 0: return &v.state @@ -20400,7 +20787,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[142].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[147].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RebuildKeyspaceGraphResponse); i { case 0: return &v.state @@ -20412,7 +20799,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[143].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[148].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RebuildVSchemaGraphRequest); i { case 0: return &v.state @@ -20424,7 +20811,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[144].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[149].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RebuildVSchemaGraphResponse); i { case 0: return &v.state @@ -20436,7 +20823,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[145].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[150].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RefreshStateRequest); i { case 0: return &v.state @@ -20448,7 +20835,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[146].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[151].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RefreshStateResponse); i { case 0: return &v.state @@ -20460,7 +20847,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[147].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[152].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RefreshStateByShardRequest); i { case 0: return &v.state @@ -20472,7 +20859,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[148].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[153].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RefreshStateByShardResponse); i { case 0: return &v.state @@ -20484,7 +20871,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[149].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[154].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaRequest); i { case 0: return &v.state @@ -20496,7 +20883,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[150].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[155].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaResponse); i { case 0: return &v.state @@ -20508,7 +20895,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[151].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[156].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaKeyspaceRequest); i { case 0: return &v.state @@ -20520,7 +20907,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[152].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[157].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaKeyspaceResponse); i { case 0: return &v.state @@ -20532,7 +20919,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[153].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[158].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaShardRequest); i { case 0: return &v.state @@ -20544,7 +20931,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[154].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[159].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReloadSchemaShardResponse); i { case 0: return &v.state @@ -20556,7 +20943,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[155].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[160].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveBackupRequest); i { case 0: return &v.state @@ -20568,7 +20955,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[156].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[161].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveBackupResponse); i { case 0: return &v.state @@ -20580,7 +20967,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[157].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[162].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveKeyspaceCellRequest); i { case 0: return &v.state @@ -20592,7 +20979,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[158].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[163].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveKeyspaceCellResponse); i { case 0: return &v.state @@ -20604,7 +20991,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[159].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[164].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveShardCellRequest); i { case 0: return &v.state @@ -20616,7 +21003,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[160].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[165].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RemoveShardCellResponse); i { case 0: return &v.state @@ -20628,7 +21015,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[161].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[166].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReparentTabletRequest); i { case 0: return &v.state @@ -20640,7 +21027,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[162].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[167].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReparentTabletResponse); i { case 0: return &v.state @@ -20652,7 +21039,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[163].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[168].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReshardCreateRequest); i { case 0: return &v.state @@ -20664,7 +21051,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[164].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[169].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RestoreFromBackupRequest); i { case 0: return &v.state @@ -20676,7 +21063,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[165].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[170].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RestoreFromBackupResponse); i { case 0: return &v.state @@ -20688,7 +21075,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[166].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[171].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RetrySchemaMigrationRequest); i { case 0: return &v.state @@ -20700,7 +21087,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[167].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[172].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RetrySchemaMigrationResponse); i { case 0: return &v.state @@ -20712,7 +21099,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[168].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[173].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RunHealthCheckRequest); i { case 0: return &v.state @@ -20724,7 +21111,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[169].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[174].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*RunHealthCheckResponse); i { case 0: return &v.state @@ -20736,7 +21123,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[170].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[175].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceDurabilityPolicyRequest); i { case 0: return &v.state @@ -20748,7 +21135,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[171].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[176].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceDurabilityPolicyResponse); i { case 0: return &v.state @@ -20760,7 +21147,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[172].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[177].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceShardingInfoRequest); i { case 0: return &v.state @@ -20772,7 +21159,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[173].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[178].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetKeyspaceShardingInfoResponse); i { case 0: return &v.state @@ -20784,7 +21171,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[174].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[179].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardIsPrimaryServingRequest); i { case 0: return &v.state @@ -20796,7 +21183,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[175].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[180].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardIsPrimaryServingResponse); i { case 0: return &v.state @@ -20808,7 +21195,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[176].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[181].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardTabletControlRequest); i { case 0: return &v.state @@ -20820,7 +21207,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[177].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[182].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetShardTabletControlResponse); i { case 0: return &v.state @@ -20832,7 +21219,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[178].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[183].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetWritableRequest); i { case 0: return &v.state @@ -20844,7 +21231,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[179].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[184].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SetWritableResponse); i { case 0: return &v.state @@ -20856,7 +21243,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[180].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[185].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationAddRequest); i { case 0: return &v.state @@ -20868,7 +21255,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[181].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[186].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationAddResponse); i { case 0: return &v.state @@ -20880,7 +21267,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[182].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[187].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationFixRequest); i { case 0: return &v.state @@ -20892,7 +21279,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[183].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[188].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationFixResponse); i { case 0: return &v.state @@ -20904,7 +21291,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[184].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[189].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationPositionsRequest); i { case 0: return &v.state @@ -20916,7 +21303,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[185].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[190].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationPositionsResponse); i { case 0: return &v.state @@ -20928,7 +21315,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[186].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[191].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationRemoveRequest); i { case 0: return &v.state @@ -20940,7 +21327,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[187].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[192].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ShardReplicationRemoveResponse); i { case 0: return &v.state @@ -20952,7 +21339,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[188].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[193].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SleepTabletRequest); i { case 0: return &v.state @@ -20964,7 +21351,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[189].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[194].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SleepTabletResponse); i { case 0: return &v.state @@ -20976,7 +21363,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[190].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[195].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardAddRequest); i { case 0: return &v.state @@ -20988,7 +21375,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[191].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[196].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardAddResponse); i { case 0: return &v.state @@ -21000,7 +21387,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[192].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[197].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardDeleteRequest); i { case 0: return &v.state @@ -21012,7 +21399,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[193].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[198].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SourceShardDeleteResponse); i { case 0: return &v.state @@ -21024,7 +21411,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[194].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[199].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StartReplicationRequest); i { case 0: return &v.state @@ -21036,7 +21423,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[195].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[200].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StartReplicationResponse); i { case 0: return &v.state @@ -21048,7 +21435,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[196].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[201].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StopReplicationRequest); i { case 0: return &v.state @@ -21060,7 +21447,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[197].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[202].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*StopReplicationResponse); i { case 0: return &v.state @@ -21072,7 +21459,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[198].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[203].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TabletExternallyReparentedRequest); i { case 0: return &v.state @@ -21084,7 +21471,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[199].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[204].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*TabletExternallyReparentedResponse); i { case 0: return &v.state @@ -21096,7 +21483,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[200].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[205].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellInfoRequest); i { case 0: return &v.state @@ -21108,7 +21495,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[201].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[206].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellInfoResponse); i { case 0: return &v.state @@ -21120,7 +21507,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[202].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[207].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellsAliasRequest); i { case 0: return &v.state @@ -21132,7 +21519,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[203].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[208].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateCellsAliasResponse); i { case 0: return &v.state @@ -21144,7 +21531,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[204].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[209].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateRequest); i { case 0: return &v.state @@ -21156,7 +21543,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[205].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[210].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateResponse); i { case 0: return &v.state @@ -21168,7 +21555,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[206].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[211].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateKeyspaceRequest); i { case 0: return &v.state @@ -21180,7 +21567,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[207].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[212].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateKeyspaceResponse); i { case 0: return &v.state @@ -21192,7 +21579,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[208].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[213].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateSchemaKeyspaceRequest); i { case 0: return &v.state @@ -21204,7 +21591,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[209].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[214].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateSchemaKeyspaceResponse); i { case 0: return &v.state @@ -21216,7 +21603,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[210].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[215].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateShardRequest); i { case 0: return &v.state @@ -21228,7 +21615,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[211].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[216].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateShardResponse); i { case 0: return &v.state @@ -21240,7 +21627,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[212].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[217].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionKeyspaceRequest); i { case 0: return &v.state @@ -21252,7 +21639,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[213].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[218].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionKeyspaceResponse); i { case 0: return &v.state @@ -21264,7 +21651,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[214].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[219].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionShardRequest); i { case 0: return &v.state @@ -21276,7 +21663,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[215].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[220].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVersionShardResponse); i { case 0: return &v.state @@ -21288,7 +21675,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[216].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[221].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVSchemaRequest); i { case 0: return &v.state @@ -21300,7 +21687,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[217].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[222].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ValidateVSchemaResponse); i { case 0: return &v.state @@ -21312,7 +21699,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[218].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[223].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffCreateRequest); i { case 0: return &v.state @@ -21324,7 +21711,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[219].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[224].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffCreateResponse); i { case 0: return &v.state @@ -21336,7 +21723,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[220].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[225].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffDeleteRequest); i { case 0: return &v.state @@ -21348,7 +21735,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[221].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[226].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffDeleteResponse); i { case 0: return &v.state @@ -21360,7 +21747,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[222].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[227].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffResumeRequest); i { case 0: return &v.state @@ -21372,7 +21759,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[223].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[228].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffResumeResponse); i { case 0: return &v.state @@ -21384,7 +21771,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[224].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[229].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffShowRequest); i { case 0: return &v.state @@ -21396,7 +21783,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[225].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[230].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffShowResponse); i { case 0: return &v.state @@ -21408,7 +21795,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[226].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[231].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffStopRequest); i { case 0: return &v.state @@ -21420,7 +21807,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[227].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[232].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*VDiffStopResponse); i { case 0: return &v.state @@ -21432,7 +21819,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[228].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[233].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowDeleteRequest); i { case 0: return &v.state @@ -21444,7 +21831,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[229].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[234].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowDeleteResponse); i { case 0: return &v.state @@ -21456,7 +21843,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[230].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[235].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowStatusRequest); i { case 0: return &v.state @@ -21468,7 +21855,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[231].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[236].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowStatusResponse); i { case 0: return &v.state @@ -21480,7 +21867,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[232].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[237].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowSwitchTrafficRequest); i { case 0: return &v.state @@ -21492,7 +21879,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[233].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[238].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowSwitchTrafficResponse); i { case 0: return &v.state @@ -21504,7 +21891,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[234].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[239].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowUpdateRequest); i { case 0: return &v.state @@ -21516,7 +21903,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[235].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[240].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowUpdateResponse); i { case 0: return &v.state @@ -21528,7 +21915,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[237].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[242].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_ReplicationLocation); i { case 0: return &v.state @@ -21540,7 +21927,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[238].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[243].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_ShardStream); i { case 0: return &v.state @@ -21552,7 +21939,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[239].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[244].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream); i { case 0: return &v.state @@ -21564,7 +21951,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[240].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[245].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream_CopyState); i { case 0: return &v.state @@ -21576,7 +21963,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[241].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[246].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream_Log); i { case 0: return &v.state @@ -21588,7 +21975,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[242].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[247].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Workflow_Stream_ThrottlerStatus); i { case 0: return &v.state @@ -21600,7 +21987,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[245].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[250].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ApplyVSchemaResponse_ParamList); i { case 0: return &v.state @@ -21612,7 +21999,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[254].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[259].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetSrvKeyspaceNamesResponse_NameList); i { case 0: return &v.state @@ -21624,7 +22011,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[258].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[263].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*MoveTablesCreateResponse_TabletInfo); i { case 0: return &v.state @@ -21636,7 +22023,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[268].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[273].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowDeleteResponse_TabletInfo); i { case 0: return &v.state @@ -21648,7 +22035,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[269].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[274].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowStatusResponse_TableCopyState); i { case 0: return &v.state @@ -21660,7 +22047,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[270].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[275].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowStatusResponse_ShardStreamState); i { case 0: return &v.state @@ -21672,7 +22059,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[271].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[276].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowStatusResponse_ShardStreams); i { case 0: return &v.state @@ -21684,7 +22071,7 @@ func file_vtctldata_proto_init() { return nil } } - file_vtctldata_proto_msgTypes[274].Exporter = func(v interface{}, i int) interface{} { + file_vtctldata_proto_msgTypes[279].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*WorkflowUpdateResponse_TabletInfo); i { case 0: return &v.state @@ -21703,7 +22090,7 @@ func file_vtctldata_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_vtctldata_proto_rawDesc, NumEnums: 4, - NumMessages: 275, + NumMessages: 280, NumExtensions: 0, NumServices: 0, }, diff --git a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go index 1e027e237e1..8be873b5026 100644 --- a/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go +++ b/go/vt/proto/vtctldata/vtctldata_vtproto.pb.go @@ -111,6 +111,7 @@ func (m *MaterializeSettings) CloneVT() *MaterializeSettings { DeferSecondaryKeys: m.DeferSecondaryKeys, TabletSelectionPreference: m.TabletSelectionPreference, AtomicCopy: m.AtomicCopy, + WorkflowOptions: m.WorkflowOptions.CloneVT(), } if rhs := m.TableSettings; rhs != nil { tmpContainer := make([]*TableMaterializeSettings, len(rhs)) @@ -245,6 +246,25 @@ func (m *Shard) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *WorkflowOptions) CloneVT() *WorkflowOptions { + if m == nil { + return (*WorkflowOptions)(nil) + } + r := &WorkflowOptions{ + TenantId: m.TenantId, + SourceKeyspaceAlias: m.SourceKeyspaceAlias, + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *WorkflowOptions) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *Workflow_ReplicationLocation) CloneVT() *Workflow_ReplicationLocation { if m == nil { return (*Workflow_ReplicationLocation)(nil) @@ -438,6 +458,7 @@ func (m *Workflow) CloneVT() *Workflow { WorkflowSubType: m.WorkflowSubType, MaxVReplicationTransactionLag: m.MaxVReplicationTransactionLag, DeferSecondaryKeys: m.DeferSecondaryKeys, + Options: m.Options.CloneVT(), } if rhs := m.ShardStreams; rhs != nil { tmpContainer := make(map[string]*Workflow_ShardStream, len(rhs)) @@ -531,6 +552,46 @@ func (m *AddCellsAliasResponse) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *ApplyKeyspaceRoutingRulesRequest) CloneVT() *ApplyKeyspaceRoutingRulesRequest { + if m == nil { + return (*ApplyKeyspaceRoutingRulesRequest)(nil) + } + r := &ApplyKeyspaceRoutingRulesRequest{ + KeyspaceRoutingRules: m.KeyspaceRoutingRules.CloneVT(), + SkipRebuild: m.SkipRebuild, + } + if rhs := m.RebuildCells; rhs != nil { + tmpContainer := make([]string, len(rhs)) + copy(tmpContainer, rhs) + r.RebuildCells = tmpContainer + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *ApplyKeyspaceRoutingRulesRequest) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *ApplyKeyspaceRoutingRulesResponse) CloneVT() *ApplyKeyspaceRoutingRulesResponse { + if m == nil { + return (*ApplyKeyspaceRoutingRulesResponse)(nil) + } + r := &ApplyKeyspaceRoutingRulesResponse{} + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *ApplyKeyspaceRoutingRulesResponse) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *ApplyRoutingRulesRequest) CloneVT() *ApplyRoutingRulesRequest { if m == nil { return (*ApplyRoutingRulesRequest)(nil) @@ -1890,6 +1951,40 @@ func (m *GetPermissionsResponse) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *GetKeyspaceRoutingRulesRequest) CloneVT() *GetKeyspaceRoutingRulesRequest { + if m == nil { + return (*GetKeyspaceRoutingRulesRequest)(nil) + } + r := &GetKeyspaceRoutingRulesRequest{} + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *GetKeyspaceRoutingRulesRequest) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *GetKeyspaceRoutingRulesResponse) CloneVT() *GetKeyspaceRoutingRulesResponse { + if m == nil { + return (*GetKeyspaceRoutingRulesResponse)(nil) + } + r := &GetKeyspaceRoutingRulesResponse{ + KeyspaceRoutingRules: m.KeyspaceRoutingRules.CloneVT(), + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *GetKeyspaceRoutingRulesResponse) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *GetRoutingRulesRequest) CloneVT() *GetRoutingRulesRequest { if m == nil { return (*GetRoutingRulesRequest)(nil) @@ -3117,6 +3212,7 @@ func (m *MoveTablesCreateRequest) CloneVT() *MoveTablesCreateRequest { AutoStart: m.AutoStart, NoRoutingRules: m.NoRoutingRules, AtomicCopy: m.AtomicCopy, + WorkflowOptions: m.WorkflowOptions.CloneVT(), } if rhs := m.Cells; rhs != nil { tmpContainer := make([]string, len(rhs)) @@ -5631,6 +5727,18 @@ func (m *MaterializeSettings) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.WorkflowOptions != nil { + size, err := m.WorkflowOptions.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x8a + } if m.AtomicCopy { i-- if m.AtomicCopy { @@ -6394,6 +6502,53 @@ func (m *Shard) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *WorkflowOptions) 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 *WorkflowOptions) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *WorkflowOptions) 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 len(m.SourceKeyspaceAlias) > 0 { + i -= len(m.SourceKeyspaceAlias) + copy(dAtA[i:], m.SourceKeyspaceAlias) + i = encodeVarint(dAtA, i, uint64(len(m.SourceKeyspaceAlias))) + i-- + dAtA[i] = 0x12 + } + if len(m.TenantId) > 0 { + i -= len(m.TenantId) + copy(dAtA[i:], m.TenantId) + i = encodeVarint(dAtA, i, uint64(len(m.TenantId))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *Workflow_ReplicationLocation) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -6951,6 +7106,16 @@ func (m *Workflow) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.Options != nil { + size, err := m.Options.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x52 + } if m.DeferSecondaryKeys { i-- if m.DeferSecondaryKeys { @@ -7202,6 +7367,101 @@ func (m *AddCellsAliasResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *ApplyKeyspaceRoutingRulesRequest) 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 *ApplyKeyspaceRoutingRulesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ApplyKeyspaceRoutingRulesRequest) 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 len(m.RebuildCells) > 0 { + for iNdEx := len(m.RebuildCells) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.RebuildCells[iNdEx]) + copy(dAtA[i:], m.RebuildCells[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.RebuildCells[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if m.SkipRebuild { + i-- + if m.SkipRebuild { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } + if m.KeyspaceRoutingRules != nil { + size, err := m.KeyspaceRoutingRules.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 *ApplyKeyspaceRoutingRulesResponse) 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 *ApplyKeyspaceRoutingRulesResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ApplyKeyspaceRoutingRulesResponse) 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) + } + return len(dAtA) - i, nil +} + func (m *ApplyRoutingRulesRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -10664,6 +10924,82 @@ func (m *GetPermissionsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error return len(dAtA) - i, nil } +func (m *GetKeyspaceRoutingRulesRequest) 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 *GetKeyspaceRoutingRulesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetKeyspaceRoutingRulesRequest) 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) + } + return len(dAtA) - i, nil +} + +func (m *GetKeyspaceRoutingRulesResponse) 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 *GetKeyspaceRoutingRulesResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *GetKeyspaceRoutingRulesResponse) 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.KeyspaceRoutingRules != nil { + size, err := m.KeyspaceRoutingRules.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 *GetRoutingRulesRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil @@ -13744,6 +14080,18 @@ func (m *MoveTablesCreateRequest) MarshalToSizedBufferVT(dAtA []byte) (int, erro i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.WorkflowOptions != nil { + size, err := m.WorkflowOptions.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0xa2 + } if m.AtomicCopy { i-- if m.AtomicCopy { @@ -20013,6 +20361,10 @@ func (m *MaterializeSettings) SizeVT() (n int) { if m.AtomicCopy { n += 3 } + if m.WorkflowOptions != nil { + l = m.WorkflowOptions.SizeVT() + n += 2 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -20264,6 +20616,24 @@ func (m *Shard) SizeVT() (n int) { return n } +func (m *WorkflowOptions) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.TenantId) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.SourceKeyspaceAlias) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + func (m *Workflow_ReplicationLocation) SizeVT() (n int) { if m == nil { return 0 @@ -20533,6 +20903,10 @@ func (m *Workflow) SizeVT() (n int) { if m.DeferSecondaryKeys { n += 2 } + if m.Options != nil { + l = m.Options.SizeVT() + n += 1 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -20595,6 +20969,39 @@ func (m *AddCellsAliasResponse) SizeVT() (n int) { return n } +func (m *ApplyKeyspaceRoutingRulesRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.KeyspaceRoutingRules != nil { + l = m.KeyspaceRoutingRules.SizeVT() + n += 1 + l + sov(uint64(l)) + } + if m.SkipRebuild { + n += 2 + } + if len(m.RebuildCells) > 0 { + for _, s := range m.RebuildCells { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + +func (m *ApplyKeyspaceRoutingRulesResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + func (m *ApplyRoutingRulesRequest) SizeVT() (n int) { if m == nil { return 0 @@ -21843,6 +22250,30 @@ func (m *GetPermissionsResponse) SizeVT() (n int) { return n } +func (m *GetKeyspaceRoutingRulesRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *GetKeyspaceRoutingRulesResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.KeyspaceRoutingRules != nil { + l = m.KeyspaceRoutingRules.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + func (m *GetRoutingRulesRequest) SizeVT() (n int) { if m == nil { return 0 @@ -23064,6 +23495,10 @@ func (m *MoveTablesCreateRequest) SizeVT() (n int) { if m.AtomicCopy { n += 3 } + if m.WorkflowOptions != nil { + l = m.WorkflowOptions.SizeVT() + n += 2 + l + sov(uint64(l)) + } n += len(m.unknownFields) return n } @@ -26137,6 +26572,42 @@ func (m *MaterializeSettings) UnmarshalVT(dAtA []byte) error { } } m.AtomicCopy = bool(v != 0) + case 17: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowOptions", 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.WorkflowOptions == nil { + m.WorkflowOptions = &WorkflowOptions{} + } + if err := m.WorkflowOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -28004,6 +28475,121 @@ func (m *Shard) UnmarshalVT(dAtA []byte) error { } return nil } +func (m *WorkflowOptions) 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: WorkflowOptions: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WorkflowOptions: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TenantId", 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.TenantId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceKeyspaceAlias", 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.SourceKeyspaceAlias = string(dAtA[iNdEx:postIndex]) + 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 *Workflow_ReplicationLocation) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -29884,6 +30470,42 @@ func (m *Workflow) UnmarshalVT(dAtA []byte) error { } } m.DeferSecondaryKeys = bool(v != 0) + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Options", 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.Options == nil { + m.Options = &WorkflowOptions{} + } + if err := m.Options.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -30242,7 +30864,7 @@ func (m *AddCellsAliasResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *ApplyRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { +func (m *ApplyKeyspaceRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -30265,15 +30887,205 @@ func (m *ApplyRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ApplyRoutingRulesRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ApplyKeyspaceRoutingRulesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ApplyRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ApplyKeyspaceRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RoutingRules", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field KeyspaceRoutingRules", 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.KeyspaceRoutingRules == nil { + m.KeyspaceRoutingRules = &vschema.KeyspaceRoutingRules{} + } + if err := m.KeyspaceRoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipRebuild", 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.SkipRebuild = bool(v != 0) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RebuildCells", 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.RebuildCells = append(m.RebuildCells, string(dAtA[iNdEx:postIndex])) + 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 *ApplyKeyspaceRoutingRulesResponse) 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: ApplyKeyspaceRoutingRulesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplyKeyspaceRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + 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 *ApplyRoutingRulesRequest) 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: ApplyRoutingRulesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ApplyRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RoutingRules", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -37733,17 +38545,327 @@ func (m *GetFullStatusRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetFullStatusRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetFullStatusRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetFullStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", 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.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} + } + if err := m.TabletAlias.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 *GetFullStatusResponse) 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: GetFullStatusResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetFullStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", 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.Status == nil { + m.Status = &replicationdata.FullStatus{} + } + if err := m.Status.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 *GetKeyspacesRequest) 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: GetKeyspacesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetKeyspacesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + 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 *GetKeyspacesResponse) 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: GetKeyspacesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetKeyspacesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Keyspaces", 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 + } + m.Keyspaces = append(m.Keyspaces, &Keyspace{}) + if err := m.Keyspaces[len(m.Keyspaces)-1].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 *GetKeyspaceRequest) 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: GetKeyspaceRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetFullStatusRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -37753,27 +38875,23 @@ func (m *GetFullStatusRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLength } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - if m.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} - } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -37797,7 +38915,7 @@ func (m *GetFullStatusRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetFullStatusResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -37820,15 +38938,15 @@ func (m *GetFullStatusResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetFullStatusResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspaceResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetFullStatusResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -37855,10 +38973,10 @@ func (m *GetFullStatusResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Status == nil { - m.Status = &replicationdata.FullStatus{} + if m.Keyspace == nil { + m.Keyspace = &Keyspace{} } - if err := m.Status.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -37884,58 +39002,7 @@ func (m *GetFullStatusResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetKeyspacesRequest) 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: GetKeyspacesRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetKeyspacesRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - 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 *GetKeyspacesResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -37958,15 +39025,15 @@ func (m *GetKeyspacesResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetKeyspacesResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetPermissionsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetKeyspacesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetPermissionsRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspaces", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -37993,93 +39060,12 @@ func (m *GetKeyspacesResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keyspaces = append(m.Keyspaces, &Keyspace{}) - if err := m.Keyspaces[len(m.Keyspaces)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err + if m.TabletAlias == nil { + m.TabletAlias = &topodata.TabletAlias{} } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skip(dAtA[iNdEx:]) - if err != nil { + if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); 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 *GetKeyspaceRequest) 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: GetKeyspaceRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: GetKeyspaceRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", 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.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -38103,7 +39089,7 @@ func (m *GetKeyspaceRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -38126,15 +39112,15 @@ func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetKeyspaceResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetPermissionsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetKeyspaceResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetPermissionsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Permissions", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -38161,10 +39147,10 @@ func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Keyspace == nil { - m.Keyspace = &Keyspace{} + if m.Permissions == nil { + m.Permissions = &tabletmanagerdata.Permissions{} } - if err := m.Keyspace.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Permissions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -38190,7 +39176,7 @@ func (m *GetKeyspaceResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspaceRoutingRulesRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -38213,48 +39199,12 @@ func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetPermissionsRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspaceRoutingRulesRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetPermissionsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetKeyspaceRoutingRulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletAlias", 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.TabletAlias == nil { - m.TabletAlias = &topodata.TabletAlias{} - } - if err := m.TabletAlias.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -38277,7 +39227,7 @@ func (m *GetPermissionsRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { +func (m *GetKeyspaceRoutingRulesResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -38300,15 +39250,15 @@ func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetPermissionsResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetKeyspaceRoutingRulesResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetPermissionsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetKeyspaceRoutingRulesResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Permissions", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field KeyspaceRoutingRules", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -38335,10 +39285,10 @@ func (m *GetPermissionsResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Permissions == nil { - m.Permissions = &tabletmanagerdata.Permissions{} + if m.KeyspaceRoutingRules == nil { + m.KeyspaceRoutingRules = &vschema.KeyspaceRoutingRules{} } - if err := m.Permissions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.KeyspaceRoutingRules.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -46141,6 +47091,42 @@ func (m *MoveTablesCreateRequest) UnmarshalVT(dAtA []byte) error { } } m.AtomicCopy = bool(v != 0) + case 20: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowOptions", 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.WorkflowOptions == nil { + m.WorkflowOptions = &WorkflowOptions{} + } + if err := m.WorkflowOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) diff --git a/go/vt/proto/vtctlservice/vtctlservice.pb.go b/go/vt/proto/vtctlservice/vtctlservice.pb.go index 2e05d9455c9..b4422161f7a 100644 --- a/go/vt/proto/vtctlservice/vtctlservice.pb.go +++ b/go/vt/proto/vtctlservice/vtctlservice.pb.go @@ -51,7 +51,7 @@ var file_vtctlservice_proto_rawDesc = []byte{ 0x61, 0x6e, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x56, 0x74, 0x63, 0x74, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x32, 0xec, 0x52, 0x0a, 0x06, 0x56, 0x74, 0x63, 0x74, 0x6c, + 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x32, 0xda, 0x54, 0x0a, 0x06, 0x56, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x12, 0x4e, 0x0a, 0x0b, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x64, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, @@ -73,651 +73,666 @@ var file_vtctlservice_proto_rawDesc = []byte{ 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 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, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x41, 0x70, 0x70, - 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, - 0x6c, 0x65, 0x73, 0x12, 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 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, 0x1a, 0x29, 0x2e, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x19, 0x41, 0x70, 0x70, + 0x6c, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, + 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x2b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x41, 0x70, 0x70, 0x6c, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, + 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x41, 0x70, - 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x41, 0x0a, - 0x06, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x12, 0x18, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x19, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, - 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, - 0x12, 0x4b, 0x0a, 0x0b, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, - 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, - 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x6c, 0x0a, - 0x15, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x28, 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, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x43, - 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, - 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x43, 0x6c, - 0x65, 0x61, 0x6e, 0x75, 0x70, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x41, 0x0a, 0x06, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x12, 0x18, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4b, 0x0a, 0x0b, 0x42, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x6c, 0x0a, 0x15, 0x43, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x27, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 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, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x43, 0x6c, 0x65, 0x61, 0x6e, 0x75, 0x70, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, 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, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x17, 0x43, - 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 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, 0x1a, 0x2a, 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, 0x22, 0x00, 0x12, - 0x57, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, - 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, - 0x12, 0x57, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 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, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x44, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x12, 0x1f, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x45, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x6e, 0x63, 0x79, 0x52, 0x65, - 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x28, 0x2e, 0x76, 0x74, - 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 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, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x29, 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, 0x1a, 0x2a, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x20, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, + 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 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, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x20, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x73, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x72, 0x76, + 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x12, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x45, + 0x6d, 0x65, 0x72, 0x67, 0x65, 0x6e, 0x63, 0x79, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, - 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, - 0x63, 0x68, 0x41, 0x73, 0x41, 0x70, 0x70, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, - 0x41, 0x73, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, - 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, - 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, + 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x41, 0x70, + 0x70, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x41, 0x70, 0x70, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x60, + 0x0a, 0x11, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, + 0x44, 0x42, 0x41, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, + 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, + 0x12, 0x4c, 0x0a, 0x0b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x12, + 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x0b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6f, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, - 0x75, 0x6c, 0x74, 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x12, 0x28, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, - 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x74, 0x65, 0x48, 0x6f, 0x6f, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6f, + 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x46, 0x65, + 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x12, 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, - 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x17, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x29, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 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, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x1b, 0x46, 0x6f, 0x72, - 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, - 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, - 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x47, 0x65, 0x74, - 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, - 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, + 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x46, 0x65, 0x74, 0x63, 0x68, + 0x41, 0x73, 0x44, 0x42, 0x41, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x72, 0x0a, 0x17, 0x46, 0x69, 0x6e, 0x64, 0x41, 0x6c, 0x6c, 0x53, 0x68, 0x61, 0x72, 0x64, 0x73, + 0x49, 0x6e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x29, 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, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 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, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x1b, 0x46, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, + 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x2d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x6f, 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6f, + 0x72, 0x63, 0x65, 0x43, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x73, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x42, + 0x61, 0x63, 0x6b, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x4e, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x65, - 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, - 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 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, 0x22, - 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, - 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x4b, 0x65, - 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x47, 0x65, - 0x74, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x20, 0x2e, 0x76, + 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x5d, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x65, 0x73, 0x12, 0x21, 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, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 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, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x47, + 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1f, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, 0x6c, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x75, 0x6c, + 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x73, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, + 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x50, + 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, + 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x65, 0x72, 0x6d, - 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x65, - 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, - 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, - 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, - 0x52, 0x75, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x48, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, + 0x75, 0x6c, 0x65, 0x73, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x75, + 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, + 0x09, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 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, 0x22, 0x00, 0x12, + 0x66, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x45, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x53, 0x68, + 0x61, 0x72, 0x64, 0x12, 0x1a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, + 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, 0x69, 0x6e, + 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x1a, 0x27, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x47, 0x65, 0x74, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 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, 0x22, - 0x00, 0x12, 0x66, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x45, 0x0a, 0x08, 0x47, 0x65, 0x74, - 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x69, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x6f, 0x75, 0x74, - 0x69, 0x6e, 0x67, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, - 0x1a, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x47, - 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x73, 0x12, 0x25, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 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, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, - 0x73, 0x70, 0x61, 0x63, 0x65, 0x73, 0x12, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 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, 0x22, 0x00, 0x12, - 0x6c, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, - 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x54, 0x0a, - 0x0d, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1f, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, - 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, - 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x73, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 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, 0x22, + 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x4b, 0x65, 0x79, 0x73, 0x70, + 0x61, 0x63, 0x65, 0x73, 0x12, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 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, 0x22, 0x00, 0x12, 0x6c, 0x0a, + 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x1a, + 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x47, + 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1f, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, + 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x72, 0x76, 0x56, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x73, 0x12, 0x20, 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, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, - 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x73, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, - 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, - 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x50, 0x61, - 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, - 0x79, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x4b, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x76, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x47, 0x65, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x73, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, + 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, + 0x50, 0x61, 0x74, 0x68, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x50, 0x61, 0x74, 0x68, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x6f, 0x6c, 0x6f, 0x67, 0x79, 0x50, + 0x61, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, + 0x0a, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, - 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x47, 0x65, + 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x47, 0x65, 0x74, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, - 0x49, 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, - 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, - 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x15, 0x4c, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x49, 0x6e, + 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x22, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x53, + 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, + 0x6e, 0x69, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x15, 0x4c, 0x61, 0x75, + 0x6e, 0x63, 0x68, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x27, 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, 0x12, 0x27, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 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, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, 0x4c, 0x6f, 0x6f, - 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, - 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, - 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, - 0x0a, 0x17, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x12, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, - 0x6c, 0x69, 0x7a, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x43, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 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, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, + 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x24, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, + 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x17, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x12, 0x29, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4c, + 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x45, 0x78, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x60, 0x0a, 0x11, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x12, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x4d, 0x6f, 0x75, + 0x6e, 0x74, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1f, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x67, 0x69, + 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x67, + 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5a, 0x0a, 0x0f, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, + 0x65, 0x72, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, + 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x4d, + 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x4c, 0x69, + 0x73, 0x74, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, + 0x6f, 0x75, 0x6e, 0x74, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, + 0x74, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5b, 0x0a, 0x10, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x12, 0x22, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, + 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x12, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, + 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, + 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x69, 0x6e, 0x67, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, + 0x0a, 0x14, 0x50, 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x1a, 0x27, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x14, 0x52, 0x65, 0x62, + 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, 0x61, 0x70, + 0x68, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, + 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x56, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x25, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x51, 0x0a, 0x0c, + 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x66, 0x0a, 0x13, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x1a, 0x26, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x52, 0x65, 0x6c, 0x6f, 0x61, + 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x14, 0x52, 0x65, + 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x27, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, 0x52, 0x65, + 0x6d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, + 0x12, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x00, 0x12, + 0x5a, 0x0a, 0x0f, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, + 0x6c, 0x6c, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, + 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, + 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x52, + 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, 0x20, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, + 0x61, 0x2e, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x4d, - 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x12, 0x1f, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, 0x65, - 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x52, - 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, - 0x73, 0x74, 0x65, 0x72, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, - 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, - 0x09, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x4d, 0x6f, 0x75, 0x6e, 0x74, - 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x4d, 0x6f, 0x75, 0x6e, 0x74, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x6f, - 0x75, 0x6e, 0x74, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5b, 0x0a, 0x10, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x22, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x63, - 0x0a, 0x12, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, - 0x6c, 0x65, 0x74, 0x65, 0x12, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 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, 0x1a, 0x25, 0x2e, 0x76, 0x74, 0x63, - 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0a, 0x50, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x12, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x69, - 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x69, 0x6e, 0x67, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x69, 0x0a, 0x14, 0x50, 0x6c, 0x61, 0x6e, 0x6e, 0x65, 0x64, 0x52, 0x65, 0x70, 0x61, 0x72, - 0x65, 0x6e, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, - 0x1a, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x14, 0x52, - 0x65, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x47, 0x72, - 0x61, 0x70, 0x68, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 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, 0x1a, 0x27, 0x2e, 0x76, 0x74, - 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x52, 0x65, 0x62, 0x75, 0x69, 0x6c, - 0x64, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x25, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 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, 0x00, 0x12, 0x51, - 0x0a, 0x0c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x66, 0x72, 0x65, - 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x66, 0x72, 0x65, - 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x66, 0x0a, 0x13, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, - 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x52, 0x65, 0x6c, - 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x14, - 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 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, 0x1a, 0x27, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x6f, 0x61, - 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x23, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6c, 0x6f, 0x61, 0x64, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0c, 0x52, 0x65, 0x6d, - 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x12, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, - 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x43, 0x65, - 0x6c, 0x6c, 0x12, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x25, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, - 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, - 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, 0x65, 0x6c, 0x6c, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x43, - 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, - 0x0e, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x12, - 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x61, - 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, - 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x62, 0x0a, - 0x11, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x30, - 0x01, 0x12, 0x69, 0x0a, 0x14, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 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, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x27, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, - 0x52, 0x75, 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x20, + 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, + 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 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, 0x22, 0x00, 0x30, 0x01, 0x12, + 0x69, 0x0a, 0x14, 0x52, 0x65, 0x74, 0x72, 0x79, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x69, + 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x27, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x52, 0x75, + 0x6e, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x20, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x75, 0x6e, 0x48, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x75, 0x6e, 0x48, 0x65, - 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x75, 0x6e, - 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x1b, 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, 0x12, 0x2d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 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, 0x1a, 0x2e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 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, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x18, 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x49, 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, - 0x67, 0x12, 0x2a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x2b, 0x2e, - 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x15, + 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x1b, 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, 0x12, 0x2d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x2e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x18, 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x49, + 0x73, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x6e, 0x67, 0x12, + 0x2a, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x2b, 0x2e, 0x76, 0x74, + 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x15, 0x53, 0x65, + 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x12, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 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, 0x1a, 0x28, - 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x53, 0x65, - 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x53, 0x68, - 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, - 0x64, 0x12, 0x25, 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, 0x41, 0x64, - 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 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, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x12, 0x25, 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, 0x46, 0x69, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x26, 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, 0x46, 0x69, 0x78, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x19, 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, 0x12, 0x2b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x76, + 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x53, 0x65, 0x74, 0x57, + 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x57, 0x72, 0x69, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x13, 0x53, 0x68, 0x61, 0x72, + 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x64, 0x64, 0x12, + 0x25, 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, 0x41, 0x64, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 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, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x66, 0x0a, 0x13, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x12, 0x25, 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, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 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, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x28, + 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 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, 0x52, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 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, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, - 0x68, 0x61, 0x72, 0x64, 0x41, 0x64, 0x64, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x78, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x19, 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, 0x12, 0x2b, 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, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x2c, 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, + 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x28, 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, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 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, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x53, 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x6c, 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x6c, + 0x65, 0x65, 0x70, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x41, 0x64, 0x64, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x41, 0x64, 0x64, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x41, - 0x64, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x41, 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, - 0x0a, 0x11, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x64, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, - 0x12, 0x5d, 0x0a, 0x10, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, - 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x00, 0x12, - 0x5a, 0x0a, 0x0f, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, - 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x1a, 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, 0x12, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x1a, 0x2d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x65, 0x12, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x5d, + 0x0a, 0x10, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, + 0x74, 0x61, 0x2e, 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, 0x00, 0x12, 0x5a, 0x0a, + 0x0f, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, + 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x1a, 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, 0x12, 0x2c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x43, 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, - 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, - 0x65, 0x6c, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, - 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, - 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, - 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x45, 0x0a, 0x08, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1a, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 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, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x22, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 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, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x28, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 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, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x56, 0x61, 0x6c, 0x69, 0x64, - 0x61, 0x74, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1f, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x72, 0x0a, - 0x17, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, - 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x29, 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, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x2a, 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, 0x22, - 0x00, 0x12, 0x69, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x26, 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, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x27, 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, 0x53, 0x68, 0x61, - 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, - 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, - 0x21, 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, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x22, 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, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x53, 0x68, 0x6f, 0x77, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x44, 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, 0x6f, 0x70, 0x12, - 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, - 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, - 0x6f, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x20, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5d, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, + 0x69, 0x61, 0x73, 0x12, 0x22, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, 0x6c, 0x69, 0x61, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x41, + 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x45, + 0x0a, 0x08, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1a, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 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, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x10, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x22, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x16, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x28, + 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 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, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0d, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1f, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x17, 0x56, + 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4b, 0x65, + 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x29, 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, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2a, 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, 0x22, 0x00, 0x12, + 0x69, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x26, 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, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x27, 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, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0f, 0x56, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x56, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x22, 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, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0b, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, + 0x65, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x1d, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x09, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, + 0x68, 0x6f, 0x77, 0x12, 0x1b, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, + 0x66, 0x66, 0x53, 0x68, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x48, 0x0a, 0x09, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, 0x6f, 0x70, 0x12, 0x1b, 0x2e, + 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, + 0x74, 0x6f, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x76, 0x74, 0x63, + 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x53, 0x74, 0x6f, 0x70, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x20, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x20, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, + 0x6c, 0x6f, 0x77, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x20, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x22, 0x00, 0x12, 0x6c, - 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x77, 0x69, 0x74, 0x63, 0x68, - 0x54, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x12, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, - 0x61, 0x74, 0x61, 0x2e, 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, - 0x1a, 0x28, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x20, + 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x15, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x77, 0x69, 0x74, 0x63, 0x68, 0x54, 0x72, + 0x61, 0x66, 0x66, 0x69, 0x63, 0x12, 0x27, 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, + 0x61, 0x2e, 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, 0x1a, 0x28, + 0x2e, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x64, 0x61, 0x74, 0x61, 0x2e, 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, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x20, 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, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 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, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x21, 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, 0x22, 0x00, 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, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6c, 0x6f, 0x77, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 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, 0x76, 0x74, 0x63, 0x74, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var file_vtctlservice_proto_goTypes = []interface{}{ @@ -726,225 +741,229 @@ var file_vtctlservice_proto_goTypes = []interface{}{ (*vtctldata.AddCellsAliasRequest)(nil), // 2: vtctldata.AddCellsAliasRequest (*vtctldata.ApplyRoutingRulesRequest)(nil), // 3: vtctldata.ApplyRoutingRulesRequest (*vtctldata.ApplySchemaRequest)(nil), // 4: vtctldata.ApplySchemaRequest - (*vtctldata.ApplyShardRoutingRulesRequest)(nil), // 5: vtctldata.ApplyShardRoutingRulesRequest - (*vtctldata.ApplyVSchemaRequest)(nil), // 6: vtctldata.ApplyVSchemaRequest - (*vtctldata.BackupRequest)(nil), // 7: vtctldata.BackupRequest - (*vtctldata.BackupShardRequest)(nil), // 8: vtctldata.BackupShardRequest - (*vtctldata.CancelSchemaMigrationRequest)(nil), // 9: vtctldata.CancelSchemaMigrationRequest - (*vtctldata.ChangeTabletTypeRequest)(nil), // 10: vtctldata.ChangeTabletTypeRequest - (*vtctldata.CleanupSchemaMigrationRequest)(nil), // 11: vtctldata.CleanupSchemaMigrationRequest - (*vtctldata.CompleteSchemaMigrationRequest)(nil), // 12: vtctldata.CompleteSchemaMigrationRequest - (*vtctldata.CreateKeyspaceRequest)(nil), // 13: vtctldata.CreateKeyspaceRequest - (*vtctldata.CreateShardRequest)(nil), // 14: vtctldata.CreateShardRequest - (*vtctldata.DeleteCellInfoRequest)(nil), // 15: vtctldata.DeleteCellInfoRequest - (*vtctldata.DeleteCellsAliasRequest)(nil), // 16: vtctldata.DeleteCellsAliasRequest - (*vtctldata.DeleteKeyspaceRequest)(nil), // 17: vtctldata.DeleteKeyspaceRequest - (*vtctldata.DeleteShardsRequest)(nil), // 18: vtctldata.DeleteShardsRequest - (*vtctldata.DeleteSrvVSchemaRequest)(nil), // 19: vtctldata.DeleteSrvVSchemaRequest - (*vtctldata.DeleteTabletsRequest)(nil), // 20: vtctldata.DeleteTabletsRequest - (*vtctldata.EmergencyReparentShardRequest)(nil), // 21: vtctldata.EmergencyReparentShardRequest - (*vtctldata.ExecuteFetchAsAppRequest)(nil), // 22: vtctldata.ExecuteFetchAsAppRequest - (*vtctldata.ExecuteFetchAsDBARequest)(nil), // 23: vtctldata.ExecuteFetchAsDBARequest - (*vtctldata.ExecuteHookRequest)(nil), // 24: vtctldata.ExecuteHookRequest - (*vtctldata.ExecuteMultiFetchAsDBARequest)(nil), // 25: vtctldata.ExecuteMultiFetchAsDBARequest - (*vtctldata.FindAllShardsInKeyspaceRequest)(nil), // 26: vtctldata.FindAllShardsInKeyspaceRequest - (*vtctldata.ForceCutOverSchemaMigrationRequest)(nil), // 27: vtctldata.ForceCutOverSchemaMigrationRequest - (*vtctldata.GetBackupsRequest)(nil), // 28: vtctldata.GetBackupsRequest - (*vtctldata.GetCellInfoRequest)(nil), // 29: vtctldata.GetCellInfoRequest - (*vtctldata.GetCellInfoNamesRequest)(nil), // 30: vtctldata.GetCellInfoNamesRequest - (*vtctldata.GetCellsAliasesRequest)(nil), // 31: vtctldata.GetCellsAliasesRequest - (*vtctldata.GetFullStatusRequest)(nil), // 32: vtctldata.GetFullStatusRequest - (*vtctldata.GetKeyspaceRequest)(nil), // 33: vtctldata.GetKeyspaceRequest - (*vtctldata.GetKeyspacesRequest)(nil), // 34: vtctldata.GetKeyspacesRequest - (*vtctldata.GetPermissionsRequest)(nil), // 35: vtctldata.GetPermissionsRequest - (*vtctldata.GetRoutingRulesRequest)(nil), // 36: vtctldata.GetRoutingRulesRequest - (*vtctldata.GetSchemaRequest)(nil), // 37: vtctldata.GetSchemaRequest - (*vtctldata.GetSchemaMigrationsRequest)(nil), // 38: vtctldata.GetSchemaMigrationsRequest - (*vtctldata.GetShardReplicationRequest)(nil), // 39: vtctldata.GetShardReplicationRequest - (*vtctldata.GetShardRequest)(nil), // 40: vtctldata.GetShardRequest - (*vtctldata.GetShardRoutingRulesRequest)(nil), // 41: vtctldata.GetShardRoutingRulesRequest - (*vtctldata.GetSrvKeyspaceNamesRequest)(nil), // 42: vtctldata.GetSrvKeyspaceNamesRequest - (*vtctldata.GetSrvKeyspacesRequest)(nil), // 43: vtctldata.GetSrvKeyspacesRequest - (*vtctldata.UpdateThrottlerConfigRequest)(nil), // 44: vtctldata.UpdateThrottlerConfigRequest - (*vtctldata.GetSrvVSchemaRequest)(nil), // 45: vtctldata.GetSrvVSchemaRequest - (*vtctldata.GetSrvVSchemasRequest)(nil), // 46: vtctldata.GetSrvVSchemasRequest - (*vtctldata.GetTabletRequest)(nil), // 47: vtctldata.GetTabletRequest - (*vtctldata.GetTabletsRequest)(nil), // 48: vtctldata.GetTabletsRequest - (*vtctldata.GetTopologyPathRequest)(nil), // 49: vtctldata.GetTopologyPathRequest - (*vtctldata.GetVersionRequest)(nil), // 50: vtctldata.GetVersionRequest - (*vtctldata.GetVSchemaRequest)(nil), // 51: vtctldata.GetVSchemaRequest - (*vtctldata.GetWorkflowsRequest)(nil), // 52: vtctldata.GetWorkflowsRequest - (*vtctldata.InitShardPrimaryRequest)(nil), // 53: vtctldata.InitShardPrimaryRequest - (*vtctldata.LaunchSchemaMigrationRequest)(nil), // 54: vtctldata.LaunchSchemaMigrationRequest - (*vtctldata.LookupVindexCreateRequest)(nil), // 55: vtctldata.LookupVindexCreateRequest - (*vtctldata.LookupVindexExternalizeRequest)(nil), // 56: vtctldata.LookupVindexExternalizeRequest - (*vtctldata.MaterializeCreateRequest)(nil), // 57: vtctldata.MaterializeCreateRequest - (*vtctldata.MigrateCreateRequest)(nil), // 58: vtctldata.MigrateCreateRequest - (*vtctldata.MountRegisterRequest)(nil), // 59: vtctldata.MountRegisterRequest - (*vtctldata.MountUnregisterRequest)(nil), // 60: vtctldata.MountUnregisterRequest - (*vtctldata.MountShowRequest)(nil), // 61: vtctldata.MountShowRequest - (*vtctldata.MountListRequest)(nil), // 62: vtctldata.MountListRequest - (*vtctldata.MoveTablesCreateRequest)(nil), // 63: vtctldata.MoveTablesCreateRequest - (*vtctldata.MoveTablesCompleteRequest)(nil), // 64: vtctldata.MoveTablesCompleteRequest - (*vtctldata.PingTabletRequest)(nil), // 65: vtctldata.PingTabletRequest - (*vtctldata.PlannedReparentShardRequest)(nil), // 66: vtctldata.PlannedReparentShardRequest - (*vtctldata.RebuildKeyspaceGraphRequest)(nil), // 67: vtctldata.RebuildKeyspaceGraphRequest - (*vtctldata.RebuildVSchemaGraphRequest)(nil), // 68: vtctldata.RebuildVSchemaGraphRequest - (*vtctldata.RefreshStateRequest)(nil), // 69: vtctldata.RefreshStateRequest - (*vtctldata.RefreshStateByShardRequest)(nil), // 70: vtctldata.RefreshStateByShardRequest - (*vtctldata.ReloadSchemaRequest)(nil), // 71: vtctldata.ReloadSchemaRequest - (*vtctldata.ReloadSchemaKeyspaceRequest)(nil), // 72: vtctldata.ReloadSchemaKeyspaceRequest - (*vtctldata.ReloadSchemaShardRequest)(nil), // 73: vtctldata.ReloadSchemaShardRequest - (*vtctldata.RemoveBackupRequest)(nil), // 74: vtctldata.RemoveBackupRequest - (*vtctldata.RemoveKeyspaceCellRequest)(nil), // 75: vtctldata.RemoveKeyspaceCellRequest - (*vtctldata.RemoveShardCellRequest)(nil), // 76: vtctldata.RemoveShardCellRequest - (*vtctldata.ReparentTabletRequest)(nil), // 77: vtctldata.ReparentTabletRequest - (*vtctldata.ReshardCreateRequest)(nil), // 78: vtctldata.ReshardCreateRequest - (*vtctldata.RestoreFromBackupRequest)(nil), // 79: vtctldata.RestoreFromBackupRequest - (*vtctldata.RetrySchemaMigrationRequest)(nil), // 80: vtctldata.RetrySchemaMigrationRequest - (*vtctldata.RunHealthCheckRequest)(nil), // 81: vtctldata.RunHealthCheckRequest - (*vtctldata.SetKeyspaceDurabilityPolicyRequest)(nil), // 82: vtctldata.SetKeyspaceDurabilityPolicyRequest - (*vtctldata.SetShardIsPrimaryServingRequest)(nil), // 83: vtctldata.SetShardIsPrimaryServingRequest - (*vtctldata.SetShardTabletControlRequest)(nil), // 84: vtctldata.SetShardTabletControlRequest - (*vtctldata.SetWritableRequest)(nil), // 85: vtctldata.SetWritableRequest - (*vtctldata.ShardReplicationAddRequest)(nil), // 86: vtctldata.ShardReplicationAddRequest - (*vtctldata.ShardReplicationFixRequest)(nil), // 87: vtctldata.ShardReplicationFixRequest - (*vtctldata.ShardReplicationPositionsRequest)(nil), // 88: vtctldata.ShardReplicationPositionsRequest - (*vtctldata.ShardReplicationRemoveRequest)(nil), // 89: vtctldata.ShardReplicationRemoveRequest - (*vtctldata.SleepTabletRequest)(nil), // 90: vtctldata.SleepTabletRequest - (*vtctldata.SourceShardAddRequest)(nil), // 91: vtctldata.SourceShardAddRequest - (*vtctldata.SourceShardDeleteRequest)(nil), // 92: vtctldata.SourceShardDeleteRequest - (*vtctldata.StartReplicationRequest)(nil), // 93: vtctldata.StartReplicationRequest - (*vtctldata.StopReplicationRequest)(nil), // 94: vtctldata.StopReplicationRequest - (*vtctldata.TabletExternallyReparentedRequest)(nil), // 95: vtctldata.TabletExternallyReparentedRequest - (*vtctldata.UpdateCellInfoRequest)(nil), // 96: vtctldata.UpdateCellInfoRequest - (*vtctldata.UpdateCellsAliasRequest)(nil), // 97: vtctldata.UpdateCellsAliasRequest - (*vtctldata.ValidateRequest)(nil), // 98: vtctldata.ValidateRequest - (*vtctldata.ValidateKeyspaceRequest)(nil), // 99: vtctldata.ValidateKeyspaceRequest - (*vtctldata.ValidateSchemaKeyspaceRequest)(nil), // 100: vtctldata.ValidateSchemaKeyspaceRequest - (*vtctldata.ValidateShardRequest)(nil), // 101: vtctldata.ValidateShardRequest - (*vtctldata.ValidateVersionKeyspaceRequest)(nil), // 102: vtctldata.ValidateVersionKeyspaceRequest - (*vtctldata.ValidateVersionShardRequest)(nil), // 103: vtctldata.ValidateVersionShardRequest - (*vtctldata.ValidateVSchemaRequest)(nil), // 104: vtctldata.ValidateVSchemaRequest - (*vtctldata.VDiffCreateRequest)(nil), // 105: vtctldata.VDiffCreateRequest - (*vtctldata.VDiffDeleteRequest)(nil), // 106: vtctldata.VDiffDeleteRequest - (*vtctldata.VDiffResumeRequest)(nil), // 107: vtctldata.VDiffResumeRequest - (*vtctldata.VDiffShowRequest)(nil), // 108: vtctldata.VDiffShowRequest - (*vtctldata.VDiffStopRequest)(nil), // 109: vtctldata.VDiffStopRequest - (*vtctldata.WorkflowDeleteRequest)(nil), // 110: vtctldata.WorkflowDeleteRequest - (*vtctldata.WorkflowStatusRequest)(nil), // 111: vtctldata.WorkflowStatusRequest - (*vtctldata.WorkflowSwitchTrafficRequest)(nil), // 112: vtctldata.WorkflowSwitchTrafficRequest - (*vtctldata.WorkflowUpdateRequest)(nil), // 113: vtctldata.WorkflowUpdateRequest - (*vtctldata.ExecuteVtctlCommandResponse)(nil), // 114: vtctldata.ExecuteVtctlCommandResponse - (*vtctldata.AddCellInfoResponse)(nil), // 115: vtctldata.AddCellInfoResponse - (*vtctldata.AddCellsAliasResponse)(nil), // 116: vtctldata.AddCellsAliasResponse - (*vtctldata.ApplyRoutingRulesResponse)(nil), // 117: vtctldata.ApplyRoutingRulesResponse - (*vtctldata.ApplySchemaResponse)(nil), // 118: vtctldata.ApplySchemaResponse - (*vtctldata.ApplyShardRoutingRulesResponse)(nil), // 119: vtctldata.ApplyShardRoutingRulesResponse - (*vtctldata.ApplyVSchemaResponse)(nil), // 120: vtctldata.ApplyVSchemaResponse - (*vtctldata.BackupResponse)(nil), // 121: vtctldata.BackupResponse - (*vtctldata.CancelSchemaMigrationResponse)(nil), // 122: vtctldata.CancelSchemaMigrationResponse - (*vtctldata.ChangeTabletTypeResponse)(nil), // 123: vtctldata.ChangeTabletTypeResponse - (*vtctldata.CleanupSchemaMigrationResponse)(nil), // 124: vtctldata.CleanupSchemaMigrationResponse - (*vtctldata.CompleteSchemaMigrationResponse)(nil), // 125: vtctldata.CompleteSchemaMigrationResponse - (*vtctldata.CreateKeyspaceResponse)(nil), // 126: vtctldata.CreateKeyspaceResponse - (*vtctldata.CreateShardResponse)(nil), // 127: vtctldata.CreateShardResponse - (*vtctldata.DeleteCellInfoResponse)(nil), // 128: vtctldata.DeleteCellInfoResponse - (*vtctldata.DeleteCellsAliasResponse)(nil), // 129: vtctldata.DeleteCellsAliasResponse - (*vtctldata.DeleteKeyspaceResponse)(nil), // 130: vtctldata.DeleteKeyspaceResponse - (*vtctldata.DeleteShardsResponse)(nil), // 131: vtctldata.DeleteShardsResponse - (*vtctldata.DeleteSrvVSchemaResponse)(nil), // 132: vtctldata.DeleteSrvVSchemaResponse - (*vtctldata.DeleteTabletsResponse)(nil), // 133: vtctldata.DeleteTabletsResponse - (*vtctldata.EmergencyReparentShardResponse)(nil), // 134: vtctldata.EmergencyReparentShardResponse - (*vtctldata.ExecuteFetchAsAppResponse)(nil), // 135: vtctldata.ExecuteFetchAsAppResponse - (*vtctldata.ExecuteFetchAsDBAResponse)(nil), // 136: vtctldata.ExecuteFetchAsDBAResponse - (*vtctldata.ExecuteHookResponse)(nil), // 137: vtctldata.ExecuteHookResponse - (*vtctldata.ExecuteMultiFetchAsDBAResponse)(nil), // 138: vtctldata.ExecuteMultiFetchAsDBAResponse - (*vtctldata.FindAllShardsInKeyspaceResponse)(nil), // 139: vtctldata.FindAllShardsInKeyspaceResponse - (*vtctldata.ForceCutOverSchemaMigrationResponse)(nil), // 140: vtctldata.ForceCutOverSchemaMigrationResponse - (*vtctldata.GetBackupsResponse)(nil), // 141: vtctldata.GetBackupsResponse - (*vtctldata.GetCellInfoResponse)(nil), // 142: vtctldata.GetCellInfoResponse - (*vtctldata.GetCellInfoNamesResponse)(nil), // 143: vtctldata.GetCellInfoNamesResponse - (*vtctldata.GetCellsAliasesResponse)(nil), // 144: vtctldata.GetCellsAliasesResponse - (*vtctldata.GetFullStatusResponse)(nil), // 145: vtctldata.GetFullStatusResponse - (*vtctldata.GetKeyspaceResponse)(nil), // 146: vtctldata.GetKeyspaceResponse - (*vtctldata.GetKeyspacesResponse)(nil), // 147: vtctldata.GetKeyspacesResponse - (*vtctldata.GetPermissionsResponse)(nil), // 148: vtctldata.GetPermissionsResponse - (*vtctldata.GetRoutingRulesResponse)(nil), // 149: vtctldata.GetRoutingRulesResponse - (*vtctldata.GetSchemaResponse)(nil), // 150: vtctldata.GetSchemaResponse - (*vtctldata.GetSchemaMigrationsResponse)(nil), // 151: vtctldata.GetSchemaMigrationsResponse - (*vtctldata.GetShardReplicationResponse)(nil), // 152: vtctldata.GetShardReplicationResponse - (*vtctldata.GetShardResponse)(nil), // 153: vtctldata.GetShardResponse - (*vtctldata.GetShardRoutingRulesResponse)(nil), // 154: vtctldata.GetShardRoutingRulesResponse - (*vtctldata.GetSrvKeyspaceNamesResponse)(nil), // 155: vtctldata.GetSrvKeyspaceNamesResponse - (*vtctldata.GetSrvKeyspacesResponse)(nil), // 156: vtctldata.GetSrvKeyspacesResponse - (*vtctldata.UpdateThrottlerConfigResponse)(nil), // 157: vtctldata.UpdateThrottlerConfigResponse - (*vtctldata.GetSrvVSchemaResponse)(nil), // 158: vtctldata.GetSrvVSchemaResponse - (*vtctldata.GetSrvVSchemasResponse)(nil), // 159: vtctldata.GetSrvVSchemasResponse - (*vtctldata.GetTabletResponse)(nil), // 160: vtctldata.GetTabletResponse - (*vtctldata.GetTabletsResponse)(nil), // 161: vtctldata.GetTabletsResponse - (*vtctldata.GetTopologyPathResponse)(nil), // 162: vtctldata.GetTopologyPathResponse - (*vtctldata.GetVersionResponse)(nil), // 163: vtctldata.GetVersionResponse - (*vtctldata.GetVSchemaResponse)(nil), // 164: vtctldata.GetVSchemaResponse - (*vtctldata.GetWorkflowsResponse)(nil), // 165: vtctldata.GetWorkflowsResponse - (*vtctldata.InitShardPrimaryResponse)(nil), // 166: vtctldata.InitShardPrimaryResponse - (*vtctldata.LaunchSchemaMigrationResponse)(nil), // 167: vtctldata.LaunchSchemaMigrationResponse - (*vtctldata.LookupVindexCreateResponse)(nil), // 168: vtctldata.LookupVindexCreateResponse - (*vtctldata.LookupVindexExternalizeResponse)(nil), // 169: vtctldata.LookupVindexExternalizeResponse - (*vtctldata.MaterializeCreateResponse)(nil), // 170: vtctldata.MaterializeCreateResponse - (*vtctldata.WorkflowStatusResponse)(nil), // 171: vtctldata.WorkflowStatusResponse - (*vtctldata.MountRegisterResponse)(nil), // 172: vtctldata.MountRegisterResponse - (*vtctldata.MountUnregisterResponse)(nil), // 173: vtctldata.MountUnregisterResponse - (*vtctldata.MountShowResponse)(nil), // 174: vtctldata.MountShowResponse - (*vtctldata.MountListResponse)(nil), // 175: vtctldata.MountListResponse - (*vtctldata.MoveTablesCompleteResponse)(nil), // 176: vtctldata.MoveTablesCompleteResponse - (*vtctldata.PingTabletResponse)(nil), // 177: vtctldata.PingTabletResponse - (*vtctldata.PlannedReparentShardResponse)(nil), // 178: vtctldata.PlannedReparentShardResponse - (*vtctldata.RebuildKeyspaceGraphResponse)(nil), // 179: vtctldata.RebuildKeyspaceGraphResponse - (*vtctldata.RebuildVSchemaGraphResponse)(nil), // 180: vtctldata.RebuildVSchemaGraphResponse - (*vtctldata.RefreshStateResponse)(nil), // 181: vtctldata.RefreshStateResponse - (*vtctldata.RefreshStateByShardResponse)(nil), // 182: vtctldata.RefreshStateByShardResponse - (*vtctldata.ReloadSchemaResponse)(nil), // 183: vtctldata.ReloadSchemaResponse - (*vtctldata.ReloadSchemaKeyspaceResponse)(nil), // 184: vtctldata.ReloadSchemaKeyspaceResponse - (*vtctldata.ReloadSchemaShardResponse)(nil), // 185: vtctldata.ReloadSchemaShardResponse - (*vtctldata.RemoveBackupResponse)(nil), // 186: vtctldata.RemoveBackupResponse - (*vtctldata.RemoveKeyspaceCellResponse)(nil), // 187: vtctldata.RemoveKeyspaceCellResponse - (*vtctldata.RemoveShardCellResponse)(nil), // 188: vtctldata.RemoveShardCellResponse - (*vtctldata.ReparentTabletResponse)(nil), // 189: vtctldata.ReparentTabletResponse - (*vtctldata.RestoreFromBackupResponse)(nil), // 190: vtctldata.RestoreFromBackupResponse - (*vtctldata.RetrySchemaMigrationResponse)(nil), // 191: vtctldata.RetrySchemaMigrationResponse - (*vtctldata.RunHealthCheckResponse)(nil), // 192: vtctldata.RunHealthCheckResponse - (*vtctldata.SetKeyspaceDurabilityPolicyResponse)(nil), // 193: vtctldata.SetKeyspaceDurabilityPolicyResponse - (*vtctldata.SetShardIsPrimaryServingResponse)(nil), // 194: vtctldata.SetShardIsPrimaryServingResponse - (*vtctldata.SetShardTabletControlResponse)(nil), // 195: vtctldata.SetShardTabletControlResponse - (*vtctldata.SetWritableResponse)(nil), // 196: vtctldata.SetWritableResponse - (*vtctldata.ShardReplicationAddResponse)(nil), // 197: vtctldata.ShardReplicationAddResponse - (*vtctldata.ShardReplicationFixResponse)(nil), // 198: vtctldata.ShardReplicationFixResponse - (*vtctldata.ShardReplicationPositionsResponse)(nil), // 199: vtctldata.ShardReplicationPositionsResponse - (*vtctldata.ShardReplicationRemoveResponse)(nil), // 200: vtctldata.ShardReplicationRemoveResponse - (*vtctldata.SleepTabletResponse)(nil), // 201: vtctldata.SleepTabletResponse - (*vtctldata.SourceShardAddResponse)(nil), // 202: vtctldata.SourceShardAddResponse - (*vtctldata.SourceShardDeleteResponse)(nil), // 203: vtctldata.SourceShardDeleteResponse - (*vtctldata.StartReplicationResponse)(nil), // 204: vtctldata.StartReplicationResponse - (*vtctldata.StopReplicationResponse)(nil), // 205: vtctldata.StopReplicationResponse - (*vtctldata.TabletExternallyReparentedResponse)(nil), // 206: vtctldata.TabletExternallyReparentedResponse - (*vtctldata.UpdateCellInfoResponse)(nil), // 207: vtctldata.UpdateCellInfoResponse - (*vtctldata.UpdateCellsAliasResponse)(nil), // 208: vtctldata.UpdateCellsAliasResponse - (*vtctldata.ValidateResponse)(nil), // 209: vtctldata.ValidateResponse - (*vtctldata.ValidateKeyspaceResponse)(nil), // 210: vtctldata.ValidateKeyspaceResponse - (*vtctldata.ValidateSchemaKeyspaceResponse)(nil), // 211: vtctldata.ValidateSchemaKeyspaceResponse - (*vtctldata.ValidateShardResponse)(nil), // 212: vtctldata.ValidateShardResponse - (*vtctldata.ValidateVersionKeyspaceResponse)(nil), // 213: vtctldata.ValidateVersionKeyspaceResponse - (*vtctldata.ValidateVersionShardResponse)(nil), // 214: vtctldata.ValidateVersionShardResponse - (*vtctldata.ValidateVSchemaResponse)(nil), // 215: vtctldata.ValidateVSchemaResponse - (*vtctldata.VDiffCreateResponse)(nil), // 216: vtctldata.VDiffCreateResponse - (*vtctldata.VDiffDeleteResponse)(nil), // 217: vtctldata.VDiffDeleteResponse - (*vtctldata.VDiffResumeResponse)(nil), // 218: vtctldata.VDiffResumeResponse - (*vtctldata.VDiffShowResponse)(nil), // 219: vtctldata.VDiffShowResponse - (*vtctldata.VDiffStopResponse)(nil), // 220: vtctldata.VDiffStopResponse - (*vtctldata.WorkflowDeleteResponse)(nil), // 221: vtctldata.WorkflowDeleteResponse - (*vtctldata.WorkflowSwitchTrafficResponse)(nil), // 222: vtctldata.WorkflowSwitchTrafficResponse - (*vtctldata.WorkflowUpdateResponse)(nil), // 223: vtctldata.WorkflowUpdateResponse + (*vtctldata.ApplyKeyspaceRoutingRulesRequest)(nil), // 5: vtctldata.ApplyKeyspaceRoutingRulesRequest + (*vtctldata.ApplyShardRoutingRulesRequest)(nil), // 6: vtctldata.ApplyShardRoutingRulesRequest + (*vtctldata.ApplyVSchemaRequest)(nil), // 7: vtctldata.ApplyVSchemaRequest + (*vtctldata.BackupRequest)(nil), // 8: vtctldata.BackupRequest + (*vtctldata.BackupShardRequest)(nil), // 9: vtctldata.BackupShardRequest + (*vtctldata.CancelSchemaMigrationRequest)(nil), // 10: vtctldata.CancelSchemaMigrationRequest + (*vtctldata.ChangeTabletTypeRequest)(nil), // 11: vtctldata.ChangeTabletTypeRequest + (*vtctldata.CleanupSchemaMigrationRequest)(nil), // 12: vtctldata.CleanupSchemaMigrationRequest + (*vtctldata.CompleteSchemaMigrationRequest)(nil), // 13: vtctldata.CompleteSchemaMigrationRequest + (*vtctldata.CreateKeyspaceRequest)(nil), // 14: vtctldata.CreateKeyspaceRequest + (*vtctldata.CreateShardRequest)(nil), // 15: vtctldata.CreateShardRequest + (*vtctldata.DeleteCellInfoRequest)(nil), // 16: vtctldata.DeleteCellInfoRequest + (*vtctldata.DeleteCellsAliasRequest)(nil), // 17: vtctldata.DeleteCellsAliasRequest + (*vtctldata.DeleteKeyspaceRequest)(nil), // 18: vtctldata.DeleteKeyspaceRequest + (*vtctldata.DeleteShardsRequest)(nil), // 19: vtctldata.DeleteShardsRequest + (*vtctldata.DeleteSrvVSchemaRequest)(nil), // 20: vtctldata.DeleteSrvVSchemaRequest + (*vtctldata.DeleteTabletsRequest)(nil), // 21: vtctldata.DeleteTabletsRequest + (*vtctldata.EmergencyReparentShardRequest)(nil), // 22: vtctldata.EmergencyReparentShardRequest + (*vtctldata.ExecuteFetchAsAppRequest)(nil), // 23: vtctldata.ExecuteFetchAsAppRequest + (*vtctldata.ExecuteFetchAsDBARequest)(nil), // 24: vtctldata.ExecuteFetchAsDBARequest + (*vtctldata.ExecuteHookRequest)(nil), // 25: vtctldata.ExecuteHookRequest + (*vtctldata.ExecuteMultiFetchAsDBARequest)(nil), // 26: vtctldata.ExecuteMultiFetchAsDBARequest + (*vtctldata.FindAllShardsInKeyspaceRequest)(nil), // 27: vtctldata.FindAllShardsInKeyspaceRequest + (*vtctldata.ForceCutOverSchemaMigrationRequest)(nil), // 28: vtctldata.ForceCutOverSchemaMigrationRequest + (*vtctldata.GetBackupsRequest)(nil), // 29: vtctldata.GetBackupsRequest + (*vtctldata.GetCellInfoRequest)(nil), // 30: vtctldata.GetCellInfoRequest + (*vtctldata.GetCellInfoNamesRequest)(nil), // 31: vtctldata.GetCellInfoNamesRequest + (*vtctldata.GetCellsAliasesRequest)(nil), // 32: vtctldata.GetCellsAliasesRequest + (*vtctldata.GetFullStatusRequest)(nil), // 33: vtctldata.GetFullStatusRequest + (*vtctldata.GetKeyspaceRequest)(nil), // 34: vtctldata.GetKeyspaceRequest + (*vtctldata.GetKeyspacesRequest)(nil), // 35: vtctldata.GetKeyspacesRequest + (*vtctldata.GetKeyspaceRoutingRulesRequest)(nil), // 36: vtctldata.GetKeyspaceRoutingRulesRequest + (*vtctldata.GetPermissionsRequest)(nil), // 37: vtctldata.GetPermissionsRequest + (*vtctldata.GetRoutingRulesRequest)(nil), // 38: vtctldata.GetRoutingRulesRequest + (*vtctldata.GetSchemaRequest)(nil), // 39: vtctldata.GetSchemaRequest + (*vtctldata.GetSchemaMigrationsRequest)(nil), // 40: vtctldata.GetSchemaMigrationsRequest + (*vtctldata.GetShardReplicationRequest)(nil), // 41: vtctldata.GetShardReplicationRequest + (*vtctldata.GetShardRequest)(nil), // 42: vtctldata.GetShardRequest + (*vtctldata.GetShardRoutingRulesRequest)(nil), // 43: vtctldata.GetShardRoutingRulesRequest + (*vtctldata.GetSrvKeyspaceNamesRequest)(nil), // 44: vtctldata.GetSrvKeyspaceNamesRequest + (*vtctldata.GetSrvKeyspacesRequest)(nil), // 45: vtctldata.GetSrvKeyspacesRequest + (*vtctldata.UpdateThrottlerConfigRequest)(nil), // 46: vtctldata.UpdateThrottlerConfigRequest + (*vtctldata.GetSrvVSchemaRequest)(nil), // 47: vtctldata.GetSrvVSchemaRequest + (*vtctldata.GetSrvVSchemasRequest)(nil), // 48: vtctldata.GetSrvVSchemasRequest + (*vtctldata.GetTabletRequest)(nil), // 49: vtctldata.GetTabletRequest + (*vtctldata.GetTabletsRequest)(nil), // 50: vtctldata.GetTabletsRequest + (*vtctldata.GetTopologyPathRequest)(nil), // 51: vtctldata.GetTopologyPathRequest + (*vtctldata.GetVersionRequest)(nil), // 52: vtctldata.GetVersionRequest + (*vtctldata.GetVSchemaRequest)(nil), // 53: vtctldata.GetVSchemaRequest + (*vtctldata.GetWorkflowsRequest)(nil), // 54: vtctldata.GetWorkflowsRequest + (*vtctldata.InitShardPrimaryRequest)(nil), // 55: vtctldata.InitShardPrimaryRequest + (*vtctldata.LaunchSchemaMigrationRequest)(nil), // 56: vtctldata.LaunchSchemaMigrationRequest + (*vtctldata.LookupVindexCreateRequest)(nil), // 57: vtctldata.LookupVindexCreateRequest + (*vtctldata.LookupVindexExternalizeRequest)(nil), // 58: vtctldata.LookupVindexExternalizeRequest + (*vtctldata.MaterializeCreateRequest)(nil), // 59: vtctldata.MaterializeCreateRequest + (*vtctldata.MigrateCreateRequest)(nil), // 60: vtctldata.MigrateCreateRequest + (*vtctldata.MountRegisterRequest)(nil), // 61: vtctldata.MountRegisterRequest + (*vtctldata.MountUnregisterRequest)(nil), // 62: vtctldata.MountUnregisterRequest + (*vtctldata.MountShowRequest)(nil), // 63: vtctldata.MountShowRequest + (*vtctldata.MountListRequest)(nil), // 64: vtctldata.MountListRequest + (*vtctldata.MoveTablesCreateRequest)(nil), // 65: vtctldata.MoveTablesCreateRequest + (*vtctldata.MoveTablesCompleteRequest)(nil), // 66: vtctldata.MoveTablesCompleteRequest + (*vtctldata.PingTabletRequest)(nil), // 67: vtctldata.PingTabletRequest + (*vtctldata.PlannedReparentShardRequest)(nil), // 68: vtctldata.PlannedReparentShardRequest + (*vtctldata.RebuildKeyspaceGraphRequest)(nil), // 69: vtctldata.RebuildKeyspaceGraphRequest + (*vtctldata.RebuildVSchemaGraphRequest)(nil), // 70: vtctldata.RebuildVSchemaGraphRequest + (*vtctldata.RefreshStateRequest)(nil), // 71: vtctldata.RefreshStateRequest + (*vtctldata.RefreshStateByShardRequest)(nil), // 72: vtctldata.RefreshStateByShardRequest + (*vtctldata.ReloadSchemaRequest)(nil), // 73: vtctldata.ReloadSchemaRequest + (*vtctldata.ReloadSchemaKeyspaceRequest)(nil), // 74: vtctldata.ReloadSchemaKeyspaceRequest + (*vtctldata.ReloadSchemaShardRequest)(nil), // 75: vtctldata.ReloadSchemaShardRequest + (*vtctldata.RemoveBackupRequest)(nil), // 76: vtctldata.RemoveBackupRequest + (*vtctldata.RemoveKeyspaceCellRequest)(nil), // 77: vtctldata.RemoveKeyspaceCellRequest + (*vtctldata.RemoveShardCellRequest)(nil), // 78: vtctldata.RemoveShardCellRequest + (*vtctldata.ReparentTabletRequest)(nil), // 79: vtctldata.ReparentTabletRequest + (*vtctldata.ReshardCreateRequest)(nil), // 80: vtctldata.ReshardCreateRequest + (*vtctldata.RestoreFromBackupRequest)(nil), // 81: vtctldata.RestoreFromBackupRequest + (*vtctldata.RetrySchemaMigrationRequest)(nil), // 82: vtctldata.RetrySchemaMigrationRequest + (*vtctldata.RunHealthCheckRequest)(nil), // 83: vtctldata.RunHealthCheckRequest + (*vtctldata.SetKeyspaceDurabilityPolicyRequest)(nil), // 84: vtctldata.SetKeyspaceDurabilityPolicyRequest + (*vtctldata.SetShardIsPrimaryServingRequest)(nil), // 85: vtctldata.SetShardIsPrimaryServingRequest + (*vtctldata.SetShardTabletControlRequest)(nil), // 86: vtctldata.SetShardTabletControlRequest + (*vtctldata.SetWritableRequest)(nil), // 87: vtctldata.SetWritableRequest + (*vtctldata.ShardReplicationAddRequest)(nil), // 88: vtctldata.ShardReplicationAddRequest + (*vtctldata.ShardReplicationFixRequest)(nil), // 89: vtctldata.ShardReplicationFixRequest + (*vtctldata.ShardReplicationPositionsRequest)(nil), // 90: vtctldata.ShardReplicationPositionsRequest + (*vtctldata.ShardReplicationRemoveRequest)(nil), // 91: vtctldata.ShardReplicationRemoveRequest + (*vtctldata.SleepTabletRequest)(nil), // 92: vtctldata.SleepTabletRequest + (*vtctldata.SourceShardAddRequest)(nil), // 93: vtctldata.SourceShardAddRequest + (*vtctldata.SourceShardDeleteRequest)(nil), // 94: vtctldata.SourceShardDeleteRequest + (*vtctldata.StartReplicationRequest)(nil), // 95: vtctldata.StartReplicationRequest + (*vtctldata.StopReplicationRequest)(nil), // 96: vtctldata.StopReplicationRequest + (*vtctldata.TabletExternallyReparentedRequest)(nil), // 97: vtctldata.TabletExternallyReparentedRequest + (*vtctldata.UpdateCellInfoRequest)(nil), // 98: vtctldata.UpdateCellInfoRequest + (*vtctldata.UpdateCellsAliasRequest)(nil), // 99: vtctldata.UpdateCellsAliasRequest + (*vtctldata.ValidateRequest)(nil), // 100: vtctldata.ValidateRequest + (*vtctldata.ValidateKeyspaceRequest)(nil), // 101: vtctldata.ValidateKeyspaceRequest + (*vtctldata.ValidateSchemaKeyspaceRequest)(nil), // 102: vtctldata.ValidateSchemaKeyspaceRequest + (*vtctldata.ValidateShardRequest)(nil), // 103: vtctldata.ValidateShardRequest + (*vtctldata.ValidateVersionKeyspaceRequest)(nil), // 104: vtctldata.ValidateVersionKeyspaceRequest + (*vtctldata.ValidateVersionShardRequest)(nil), // 105: vtctldata.ValidateVersionShardRequest + (*vtctldata.ValidateVSchemaRequest)(nil), // 106: vtctldata.ValidateVSchemaRequest + (*vtctldata.VDiffCreateRequest)(nil), // 107: vtctldata.VDiffCreateRequest + (*vtctldata.VDiffDeleteRequest)(nil), // 108: vtctldata.VDiffDeleteRequest + (*vtctldata.VDiffResumeRequest)(nil), // 109: vtctldata.VDiffResumeRequest + (*vtctldata.VDiffShowRequest)(nil), // 110: vtctldata.VDiffShowRequest + (*vtctldata.VDiffStopRequest)(nil), // 111: vtctldata.VDiffStopRequest + (*vtctldata.WorkflowDeleteRequest)(nil), // 112: vtctldata.WorkflowDeleteRequest + (*vtctldata.WorkflowStatusRequest)(nil), // 113: vtctldata.WorkflowStatusRequest + (*vtctldata.WorkflowSwitchTrafficRequest)(nil), // 114: vtctldata.WorkflowSwitchTrafficRequest + (*vtctldata.WorkflowUpdateRequest)(nil), // 115: vtctldata.WorkflowUpdateRequest + (*vtctldata.ExecuteVtctlCommandResponse)(nil), // 116: vtctldata.ExecuteVtctlCommandResponse + (*vtctldata.AddCellInfoResponse)(nil), // 117: vtctldata.AddCellInfoResponse + (*vtctldata.AddCellsAliasResponse)(nil), // 118: vtctldata.AddCellsAliasResponse + (*vtctldata.ApplyRoutingRulesResponse)(nil), // 119: vtctldata.ApplyRoutingRulesResponse + (*vtctldata.ApplySchemaResponse)(nil), // 120: vtctldata.ApplySchemaResponse + (*vtctldata.ApplyKeyspaceRoutingRulesResponse)(nil), // 121: vtctldata.ApplyKeyspaceRoutingRulesResponse + (*vtctldata.ApplyShardRoutingRulesResponse)(nil), // 122: vtctldata.ApplyShardRoutingRulesResponse + (*vtctldata.ApplyVSchemaResponse)(nil), // 123: vtctldata.ApplyVSchemaResponse + (*vtctldata.BackupResponse)(nil), // 124: vtctldata.BackupResponse + (*vtctldata.CancelSchemaMigrationResponse)(nil), // 125: vtctldata.CancelSchemaMigrationResponse + (*vtctldata.ChangeTabletTypeResponse)(nil), // 126: vtctldata.ChangeTabletTypeResponse + (*vtctldata.CleanupSchemaMigrationResponse)(nil), // 127: vtctldata.CleanupSchemaMigrationResponse + (*vtctldata.CompleteSchemaMigrationResponse)(nil), // 128: vtctldata.CompleteSchemaMigrationResponse + (*vtctldata.CreateKeyspaceResponse)(nil), // 129: vtctldata.CreateKeyspaceResponse + (*vtctldata.CreateShardResponse)(nil), // 130: vtctldata.CreateShardResponse + (*vtctldata.DeleteCellInfoResponse)(nil), // 131: vtctldata.DeleteCellInfoResponse + (*vtctldata.DeleteCellsAliasResponse)(nil), // 132: vtctldata.DeleteCellsAliasResponse + (*vtctldata.DeleteKeyspaceResponse)(nil), // 133: vtctldata.DeleteKeyspaceResponse + (*vtctldata.DeleteShardsResponse)(nil), // 134: vtctldata.DeleteShardsResponse + (*vtctldata.DeleteSrvVSchemaResponse)(nil), // 135: vtctldata.DeleteSrvVSchemaResponse + (*vtctldata.DeleteTabletsResponse)(nil), // 136: vtctldata.DeleteTabletsResponse + (*vtctldata.EmergencyReparentShardResponse)(nil), // 137: vtctldata.EmergencyReparentShardResponse + (*vtctldata.ExecuteFetchAsAppResponse)(nil), // 138: vtctldata.ExecuteFetchAsAppResponse + (*vtctldata.ExecuteFetchAsDBAResponse)(nil), // 139: vtctldata.ExecuteFetchAsDBAResponse + (*vtctldata.ExecuteHookResponse)(nil), // 140: vtctldata.ExecuteHookResponse + (*vtctldata.ExecuteMultiFetchAsDBAResponse)(nil), // 141: vtctldata.ExecuteMultiFetchAsDBAResponse + (*vtctldata.FindAllShardsInKeyspaceResponse)(nil), // 142: vtctldata.FindAllShardsInKeyspaceResponse + (*vtctldata.ForceCutOverSchemaMigrationResponse)(nil), // 143: vtctldata.ForceCutOverSchemaMigrationResponse + (*vtctldata.GetBackupsResponse)(nil), // 144: vtctldata.GetBackupsResponse + (*vtctldata.GetCellInfoResponse)(nil), // 145: vtctldata.GetCellInfoResponse + (*vtctldata.GetCellInfoNamesResponse)(nil), // 146: vtctldata.GetCellInfoNamesResponse + (*vtctldata.GetCellsAliasesResponse)(nil), // 147: vtctldata.GetCellsAliasesResponse + (*vtctldata.GetFullStatusResponse)(nil), // 148: vtctldata.GetFullStatusResponse + (*vtctldata.GetKeyspaceResponse)(nil), // 149: vtctldata.GetKeyspaceResponse + (*vtctldata.GetKeyspacesResponse)(nil), // 150: vtctldata.GetKeyspacesResponse + (*vtctldata.GetKeyspaceRoutingRulesResponse)(nil), // 151: vtctldata.GetKeyspaceRoutingRulesResponse + (*vtctldata.GetPermissionsResponse)(nil), // 152: vtctldata.GetPermissionsResponse + (*vtctldata.GetRoutingRulesResponse)(nil), // 153: vtctldata.GetRoutingRulesResponse + (*vtctldata.GetSchemaResponse)(nil), // 154: vtctldata.GetSchemaResponse + (*vtctldata.GetSchemaMigrationsResponse)(nil), // 155: vtctldata.GetSchemaMigrationsResponse + (*vtctldata.GetShardReplicationResponse)(nil), // 156: vtctldata.GetShardReplicationResponse + (*vtctldata.GetShardResponse)(nil), // 157: vtctldata.GetShardResponse + (*vtctldata.GetShardRoutingRulesResponse)(nil), // 158: vtctldata.GetShardRoutingRulesResponse + (*vtctldata.GetSrvKeyspaceNamesResponse)(nil), // 159: vtctldata.GetSrvKeyspaceNamesResponse + (*vtctldata.GetSrvKeyspacesResponse)(nil), // 160: vtctldata.GetSrvKeyspacesResponse + (*vtctldata.UpdateThrottlerConfigResponse)(nil), // 161: vtctldata.UpdateThrottlerConfigResponse + (*vtctldata.GetSrvVSchemaResponse)(nil), // 162: vtctldata.GetSrvVSchemaResponse + (*vtctldata.GetSrvVSchemasResponse)(nil), // 163: vtctldata.GetSrvVSchemasResponse + (*vtctldata.GetTabletResponse)(nil), // 164: vtctldata.GetTabletResponse + (*vtctldata.GetTabletsResponse)(nil), // 165: vtctldata.GetTabletsResponse + (*vtctldata.GetTopologyPathResponse)(nil), // 166: vtctldata.GetTopologyPathResponse + (*vtctldata.GetVersionResponse)(nil), // 167: vtctldata.GetVersionResponse + (*vtctldata.GetVSchemaResponse)(nil), // 168: vtctldata.GetVSchemaResponse + (*vtctldata.GetWorkflowsResponse)(nil), // 169: vtctldata.GetWorkflowsResponse + (*vtctldata.InitShardPrimaryResponse)(nil), // 170: vtctldata.InitShardPrimaryResponse + (*vtctldata.LaunchSchemaMigrationResponse)(nil), // 171: vtctldata.LaunchSchemaMigrationResponse + (*vtctldata.LookupVindexCreateResponse)(nil), // 172: vtctldata.LookupVindexCreateResponse + (*vtctldata.LookupVindexExternalizeResponse)(nil), // 173: vtctldata.LookupVindexExternalizeResponse + (*vtctldata.MaterializeCreateResponse)(nil), // 174: vtctldata.MaterializeCreateResponse + (*vtctldata.WorkflowStatusResponse)(nil), // 175: vtctldata.WorkflowStatusResponse + (*vtctldata.MountRegisterResponse)(nil), // 176: vtctldata.MountRegisterResponse + (*vtctldata.MountUnregisterResponse)(nil), // 177: vtctldata.MountUnregisterResponse + (*vtctldata.MountShowResponse)(nil), // 178: vtctldata.MountShowResponse + (*vtctldata.MountListResponse)(nil), // 179: vtctldata.MountListResponse + (*vtctldata.MoveTablesCompleteResponse)(nil), // 180: vtctldata.MoveTablesCompleteResponse + (*vtctldata.PingTabletResponse)(nil), // 181: vtctldata.PingTabletResponse + (*vtctldata.PlannedReparentShardResponse)(nil), // 182: vtctldata.PlannedReparentShardResponse + (*vtctldata.RebuildKeyspaceGraphResponse)(nil), // 183: vtctldata.RebuildKeyspaceGraphResponse + (*vtctldata.RebuildVSchemaGraphResponse)(nil), // 184: vtctldata.RebuildVSchemaGraphResponse + (*vtctldata.RefreshStateResponse)(nil), // 185: vtctldata.RefreshStateResponse + (*vtctldata.RefreshStateByShardResponse)(nil), // 186: vtctldata.RefreshStateByShardResponse + (*vtctldata.ReloadSchemaResponse)(nil), // 187: vtctldata.ReloadSchemaResponse + (*vtctldata.ReloadSchemaKeyspaceResponse)(nil), // 188: vtctldata.ReloadSchemaKeyspaceResponse + (*vtctldata.ReloadSchemaShardResponse)(nil), // 189: vtctldata.ReloadSchemaShardResponse + (*vtctldata.RemoveBackupResponse)(nil), // 190: vtctldata.RemoveBackupResponse + (*vtctldata.RemoveKeyspaceCellResponse)(nil), // 191: vtctldata.RemoveKeyspaceCellResponse + (*vtctldata.RemoveShardCellResponse)(nil), // 192: vtctldata.RemoveShardCellResponse + (*vtctldata.ReparentTabletResponse)(nil), // 193: vtctldata.ReparentTabletResponse + (*vtctldata.RestoreFromBackupResponse)(nil), // 194: vtctldata.RestoreFromBackupResponse + (*vtctldata.RetrySchemaMigrationResponse)(nil), // 195: vtctldata.RetrySchemaMigrationResponse + (*vtctldata.RunHealthCheckResponse)(nil), // 196: vtctldata.RunHealthCheckResponse + (*vtctldata.SetKeyspaceDurabilityPolicyResponse)(nil), // 197: vtctldata.SetKeyspaceDurabilityPolicyResponse + (*vtctldata.SetShardIsPrimaryServingResponse)(nil), // 198: vtctldata.SetShardIsPrimaryServingResponse + (*vtctldata.SetShardTabletControlResponse)(nil), // 199: vtctldata.SetShardTabletControlResponse + (*vtctldata.SetWritableResponse)(nil), // 200: vtctldata.SetWritableResponse + (*vtctldata.ShardReplicationAddResponse)(nil), // 201: vtctldata.ShardReplicationAddResponse + (*vtctldata.ShardReplicationFixResponse)(nil), // 202: vtctldata.ShardReplicationFixResponse + (*vtctldata.ShardReplicationPositionsResponse)(nil), // 203: vtctldata.ShardReplicationPositionsResponse + (*vtctldata.ShardReplicationRemoveResponse)(nil), // 204: vtctldata.ShardReplicationRemoveResponse + (*vtctldata.SleepTabletResponse)(nil), // 205: vtctldata.SleepTabletResponse + (*vtctldata.SourceShardAddResponse)(nil), // 206: vtctldata.SourceShardAddResponse + (*vtctldata.SourceShardDeleteResponse)(nil), // 207: vtctldata.SourceShardDeleteResponse + (*vtctldata.StartReplicationResponse)(nil), // 208: vtctldata.StartReplicationResponse + (*vtctldata.StopReplicationResponse)(nil), // 209: vtctldata.StopReplicationResponse + (*vtctldata.TabletExternallyReparentedResponse)(nil), // 210: vtctldata.TabletExternallyReparentedResponse + (*vtctldata.UpdateCellInfoResponse)(nil), // 211: vtctldata.UpdateCellInfoResponse + (*vtctldata.UpdateCellsAliasResponse)(nil), // 212: vtctldata.UpdateCellsAliasResponse + (*vtctldata.ValidateResponse)(nil), // 213: vtctldata.ValidateResponse + (*vtctldata.ValidateKeyspaceResponse)(nil), // 214: vtctldata.ValidateKeyspaceResponse + (*vtctldata.ValidateSchemaKeyspaceResponse)(nil), // 215: vtctldata.ValidateSchemaKeyspaceResponse + (*vtctldata.ValidateShardResponse)(nil), // 216: vtctldata.ValidateShardResponse + (*vtctldata.ValidateVersionKeyspaceResponse)(nil), // 217: vtctldata.ValidateVersionKeyspaceResponse + (*vtctldata.ValidateVersionShardResponse)(nil), // 218: vtctldata.ValidateVersionShardResponse + (*vtctldata.ValidateVSchemaResponse)(nil), // 219: vtctldata.ValidateVSchemaResponse + (*vtctldata.VDiffCreateResponse)(nil), // 220: vtctldata.VDiffCreateResponse + (*vtctldata.VDiffDeleteResponse)(nil), // 221: vtctldata.VDiffDeleteResponse + (*vtctldata.VDiffResumeResponse)(nil), // 222: vtctldata.VDiffResumeResponse + (*vtctldata.VDiffShowResponse)(nil), // 223: vtctldata.VDiffShowResponse + (*vtctldata.VDiffStopResponse)(nil), // 224: vtctldata.VDiffStopResponse + (*vtctldata.WorkflowDeleteResponse)(nil), // 225: vtctldata.WorkflowDeleteResponse + (*vtctldata.WorkflowSwitchTrafficResponse)(nil), // 226: vtctldata.WorkflowSwitchTrafficResponse + (*vtctldata.WorkflowUpdateResponse)(nil), // 227: vtctldata.WorkflowUpdateResponse } var file_vtctlservice_proto_depIdxs = []int32{ 0, // 0: vtctlservice.Vtctl.ExecuteVtctlCommand:input_type -> vtctldata.ExecuteVtctlCommandRequest @@ -952,231 +971,235 @@ var file_vtctlservice_proto_depIdxs = []int32{ 2, // 2: vtctlservice.Vtctld.AddCellsAlias:input_type -> vtctldata.AddCellsAliasRequest 3, // 3: vtctlservice.Vtctld.ApplyRoutingRules:input_type -> vtctldata.ApplyRoutingRulesRequest 4, // 4: vtctlservice.Vtctld.ApplySchema:input_type -> vtctldata.ApplySchemaRequest - 5, // 5: vtctlservice.Vtctld.ApplyShardRoutingRules:input_type -> vtctldata.ApplyShardRoutingRulesRequest - 6, // 6: vtctlservice.Vtctld.ApplyVSchema:input_type -> vtctldata.ApplyVSchemaRequest - 7, // 7: vtctlservice.Vtctld.Backup:input_type -> vtctldata.BackupRequest - 8, // 8: vtctlservice.Vtctld.BackupShard:input_type -> vtctldata.BackupShardRequest - 9, // 9: vtctlservice.Vtctld.CancelSchemaMigration:input_type -> vtctldata.CancelSchemaMigrationRequest - 10, // 10: vtctlservice.Vtctld.ChangeTabletType:input_type -> vtctldata.ChangeTabletTypeRequest - 11, // 11: vtctlservice.Vtctld.CleanupSchemaMigration:input_type -> vtctldata.CleanupSchemaMigrationRequest - 12, // 12: vtctlservice.Vtctld.CompleteSchemaMigration:input_type -> vtctldata.CompleteSchemaMigrationRequest - 13, // 13: vtctlservice.Vtctld.CreateKeyspace:input_type -> vtctldata.CreateKeyspaceRequest - 14, // 14: vtctlservice.Vtctld.CreateShard:input_type -> vtctldata.CreateShardRequest - 15, // 15: vtctlservice.Vtctld.DeleteCellInfo:input_type -> vtctldata.DeleteCellInfoRequest - 16, // 16: vtctlservice.Vtctld.DeleteCellsAlias:input_type -> vtctldata.DeleteCellsAliasRequest - 17, // 17: vtctlservice.Vtctld.DeleteKeyspace:input_type -> vtctldata.DeleteKeyspaceRequest - 18, // 18: vtctlservice.Vtctld.DeleteShards:input_type -> vtctldata.DeleteShardsRequest - 19, // 19: vtctlservice.Vtctld.DeleteSrvVSchema:input_type -> vtctldata.DeleteSrvVSchemaRequest - 20, // 20: vtctlservice.Vtctld.DeleteTablets:input_type -> vtctldata.DeleteTabletsRequest - 21, // 21: vtctlservice.Vtctld.EmergencyReparentShard:input_type -> vtctldata.EmergencyReparentShardRequest - 22, // 22: vtctlservice.Vtctld.ExecuteFetchAsApp:input_type -> vtctldata.ExecuteFetchAsAppRequest - 23, // 23: vtctlservice.Vtctld.ExecuteFetchAsDBA:input_type -> vtctldata.ExecuteFetchAsDBARequest - 24, // 24: vtctlservice.Vtctld.ExecuteHook:input_type -> vtctldata.ExecuteHookRequest - 25, // 25: vtctlservice.Vtctld.ExecuteMultiFetchAsDBA:input_type -> vtctldata.ExecuteMultiFetchAsDBARequest - 26, // 26: vtctlservice.Vtctld.FindAllShardsInKeyspace:input_type -> vtctldata.FindAllShardsInKeyspaceRequest - 27, // 27: vtctlservice.Vtctld.ForceCutOverSchemaMigration:input_type -> vtctldata.ForceCutOverSchemaMigrationRequest - 28, // 28: vtctlservice.Vtctld.GetBackups:input_type -> vtctldata.GetBackupsRequest - 29, // 29: vtctlservice.Vtctld.GetCellInfo:input_type -> vtctldata.GetCellInfoRequest - 30, // 30: vtctlservice.Vtctld.GetCellInfoNames:input_type -> vtctldata.GetCellInfoNamesRequest - 31, // 31: vtctlservice.Vtctld.GetCellsAliases:input_type -> vtctldata.GetCellsAliasesRequest - 32, // 32: vtctlservice.Vtctld.GetFullStatus:input_type -> vtctldata.GetFullStatusRequest - 33, // 33: vtctlservice.Vtctld.GetKeyspace:input_type -> vtctldata.GetKeyspaceRequest - 34, // 34: vtctlservice.Vtctld.GetKeyspaces:input_type -> vtctldata.GetKeyspacesRequest - 35, // 35: vtctlservice.Vtctld.GetPermissions:input_type -> vtctldata.GetPermissionsRequest - 36, // 36: vtctlservice.Vtctld.GetRoutingRules:input_type -> vtctldata.GetRoutingRulesRequest - 37, // 37: vtctlservice.Vtctld.GetSchema:input_type -> vtctldata.GetSchemaRequest - 38, // 38: vtctlservice.Vtctld.GetSchemaMigrations:input_type -> vtctldata.GetSchemaMigrationsRequest - 39, // 39: vtctlservice.Vtctld.GetShardReplication:input_type -> vtctldata.GetShardReplicationRequest - 40, // 40: vtctlservice.Vtctld.GetShard:input_type -> vtctldata.GetShardRequest - 41, // 41: vtctlservice.Vtctld.GetShardRoutingRules:input_type -> vtctldata.GetShardRoutingRulesRequest - 42, // 42: vtctlservice.Vtctld.GetSrvKeyspaceNames:input_type -> vtctldata.GetSrvKeyspaceNamesRequest - 43, // 43: vtctlservice.Vtctld.GetSrvKeyspaces:input_type -> vtctldata.GetSrvKeyspacesRequest - 44, // 44: vtctlservice.Vtctld.UpdateThrottlerConfig:input_type -> vtctldata.UpdateThrottlerConfigRequest - 45, // 45: vtctlservice.Vtctld.GetSrvVSchema:input_type -> vtctldata.GetSrvVSchemaRequest - 46, // 46: vtctlservice.Vtctld.GetSrvVSchemas:input_type -> vtctldata.GetSrvVSchemasRequest - 47, // 47: vtctlservice.Vtctld.GetTablet:input_type -> vtctldata.GetTabletRequest - 48, // 48: vtctlservice.Vtctld.GetTablets:input_type -> vtctldata.GetTabletsRequest - 49, // 49: vtctlservice.Vtctld.GetTopologyPath:input_type -> vtctldata.GetTopologyPathRequest - 50, // 50: vtctlservice.Vtctld.GetVersion:input_type -> vtctldata.GetVersionRequest - 51, // 51: vtctlservice.Vtctld.GetVSchema:input_type -> vtctldata.GetVSchemaRequest - 52, // 52: vtctlservice.Vtctld.GetWorkflows:input_type -> vtctldata.GetWorkflowsRequest - 53, // 53: vtctlservice.Vtctld.InitShardPrimary:input_type -> vtctldata.InitShardPrimaryRequest - 54, // 54: vtctlservice.Vtctld.LaunchSchemaMigration:input_type -> vtctldata.LaunchSchemaMigrationRequest - 55, // 55: vtctlservice.Vtctld.LookupVindexCreate:input_type -> vtctldata.LookupVindexCreateRequest - 56, // 56: vtctlservice.Vtctld.LookupVindexExternalize:input_type -> vtctldata.LookupVindexExternalizeRequest - 57, // 57: vtctlservice.Vtctld.MaterializeCreate:input_type -> vtctldata.MaterializeCreateRequest - 58, // 58: vtctlservice.Vtctld.MigrateCreate:input_type -> vtctldata.MigrateCreateRequest - 59, // 59: vtctlservice.Vtctld.MountRegister:input_type -> vtctldata.MountRegisterRequest - 60, // 60: vtctlservice.Vtctld.MountUnregister:input_type -> vtctldata.MountUnregisterRequest - 61, // 61: vtctlservice.Vtctld.MountShow:input_type -> vtctldata.MountShowRequest - 62, // 62: vtctlservice.Vtctld.MountList:input_type -> vtctldata.MountListRequest - 63, // 63: vtctlservice.Vtctld.MoveTablesCreate:input_type -> vtctldata.MoveTablesCreateRequest - 64, // 64: vtctlservice.Vtctld.MoveTablesComplete:input_type -> vtctldata.MoveTablesCompleteRequest - 65, // 65: vtctlservice.Vtctld.PingTablet:input_type -> vtctldata.PingTabletRequest - 66, // 66: vtctlservice.Vtctld.PlannedReparentShard:input_type -> vtctldata.PlannedReparentShardRequest - 67, // 67: vtctlservice.Vtctld.RebuildKeyspaceGraph:input_type -> vtctldata.RebuildKeyspaceGraphRequest - 68, // 68: vtctlservice.Vtctld.RebuildVSchemaGraph:input_type -> vtctldata.RebuildVSchemaGraphRequest - 69, // 69: vtctlservice.Vtctld.RefreshState:input_type -> vtctldata.RefreshStateRequest - 70, // 70: vtctlservice.Vtctld.RefreshStateByShard:input_type -> vtctldata.RefreshStateByShardRequest - 71, // 71: vtctlservice.Vtctld.ReloadSchema:input_type -> vtctldata.ReloadSchemaRequest - 72, // 72: vtctlservice.Vtctld.ReloadSchemaKeyspace:input_type -> vtctldata.ReloadSchemaKeyspaceRequest - 73, // 73: vtctlservice.Vtctld.ReloadSchemaShard:input_type -> vtctldata.ReloadSchemaShardRequest - 74, // 74: vtctlservice.Vtctld.RemoveBackup:input_type -> vtctldata.RemoveBackupRequest - 75, // 75: vtctlservice.Vtctld.RemoveKeyspaceCell:input_type -> vtctldata.RemoveKeyspaceCellRequest - 76, // 76: vtctlservice.Vtctld.RemoveShardCell:input_type -> vtctldata.RemoveShardCellRequest - 77, // 77: vtctlservice.Vtctld.ReparentTablet:input_type -> vtctldata.ReparentTabletRequest - 78, // 78: vtctlservice.Vtctld.ReshardCreate:input_type -> vtctldata.ReshardCreateRequest - 79, // 79: vtctlservice.Vtctld.RestoreFromBackup:input_type -> vtctldata.RestoreFromBackupRequest - 80, // 80: vtctlservice.Vtctld.RetrySchemaMigration:input_type -> vtctldata.RetrySchemaMigrationRequest - 81, // 81: vtctlservice.Vtctld.RunHealthCheck:input_type -> vtctldata.RunHealthCheckRequest - 82, // 82: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:input_type -> vtctldata.SetKeyspaceDurabilityPolicyRequest - 83, // 83: vtctlservice.Vtctld.SetShardIsPrimaryServing:input_type -> vtctldata.SetShardIsPrimaryServingRequest - 84, // 84: vtctlservice.Vtctld.SetShardTabletControl:input_type -> vtctldata.SetShardTabletControlRequest - 85, // 85: vtctlservice.Vtctld.SetWritable:input_type -> vtctldata.SetWritableRequest - 86, // 86: vtctlservice.Vtctld.ShardReplicationAdd:input_type -> vtctldata.ShardReplicationAddRequest - 87, // 87: vtctlservice.Vtctld.ShardReplicationFix:input_type -> vtctldata.ShardReplicationFixRequest - 88, // 88: vtctlservice.Vtctld.ShardReplicationPositions:input_type -> vtctldata.ShardReplicationPositionsRequest - 89, // 89: vtctlservice.Vtctld.ShardReplicationRemove:input_type -> vtctldata.ShardReplicationRemoveRequest - 90, // 90: vtctlservice.Vtctld.SleepTablet:input_type -> vtctldata.SleepTabletRequest - 91, // 91: vtctlservice.Vtctld.SourceShardAdd:input_type -> vtctldata.SourceShardAddRequest - 92, // 92: vtctlservice.Vtctld.SourceShardDelete:input_type -> vtctldata.SourceShardDeleteRequest - 93, // 93: vtctlservice.Vtctld.StartReplication:input_type -> vtctldata.StartReplicationRequest - 94, // 94: vtctlservice.Vtctld.StopReplication:input_type -> vtctldata.StopReplicationRequest - 95, // 95: vtctlservice.Vtctld.TabletExternallyReparented:input_type -> vtctldata.TabletExternallyReparentedRequest - 96, // 96: vtctlservice.Vtctld.UpdateCellInfo:input_type -> vtctldata.UpdateCellInfoRequest - 97, // 97: vtctlservice.Vtctld.UpdateCellsAlias:input_type -> vtctldata.UpdateCellsAliasRequest - 98, // 98: vtctlservice.Vtctld.Validate:input_type -> vtctldata.ValidateRequest - 99, // 99: vtctlservice.Vtctld.ValidateKeyspace:input_type -> vtctldata.ValidateKeyspaceRequest - 100, // 100: vtctlservice.Vtctld.ValidateSchemaKeyspace:input_type -> vtctldata.ValidateSchemaKeyspaceRequest - 101, // 101: vtctlservice.Vtctld.ValidateShard:input_type -> vtctldata.ValidateShardRequest - 102, // 102: vtctlservice.Vtctld.ValidateVersionKeyspace:input_type -> vtctldata.ValidateVersionKeyspaceRequest - 103, // 103: vtctlservice.Vtctld.ValidateVersionShard:input_type -> vtctldata.ValidateVersionShardRequest - 104, // 104: vtctlservice.Vtctld.ValidateVSchema:input_type -> vtctldata.ValidateVSchemaRequest - 105, // 105: vtctlservice.Vtctld.VDiffCreate:input_type -> vtctldata.VDiffCreateRequest - 106, // 106: vtctlservice.Vtctld.VDiffDelete:input_type -> vtctldata.VDiffDeleteRequest - 107, // 107: vtctlservice.Vtctld.VDiffResume:input_type -> vtctldata.VDiffResumeRequest - 108, // 108: vtctlservice.Vtctld.VDiffShow:input_type -> vtctldata.VDiffShowRequest - 109, // 109: vtctlservice.Vtctld.VDiffStop:input_type -> vtctldata.VDiffStopRequest - 110, // 110: vtctlservice.Vtctld.WorkflowDelete:input_type -> vtctldata.WorkflowDeleteRequest - 111, // 111: vtctlservice.Vtctld.WorkflowStatus:input_type -> vtctldata.WorkflowStatusRequest - 112, // 112: vtctlservice.Vtctld.WorkflowSwitchTraffic:input_type -> vtctldata.WorkflowSwitchTrafficRequest - 113, // 113: vtctlservice.Vtctld.WorkflowUpdate:input_type -> vtctldata.WorkflowUpdateRequest - 114, // 114: vtctlservice.Vtctl.ExecuteVtctlCommand:output_type -> vtctldata.ExecuteVtctlCommandResponse - 115, // 115: vtctlservice.Vtctld.AddCellInfo:output_type -> vtctldata.AddCellInfoResponse - 116, // 116: vtctlservice.Vtctld.AddCellsAlias:output_type -> vtctldata.AddCellsAliasResponse - 117, // 117: vtctlservice.Vtctld.ApplyRoutingRules:output_type -> vtctldata.ApplyRoutingRulesResponse - 118, // 118: vtctlservice.Vtctld.ApplySchema:output_type -> vtctldata.ApplySchemaResponse - 119, // 119: vtctlservice.Vtctld.ApplyShardRoutingRules:output_type -> vtctldata.ApplyShardRoutingRulesResponse - 120, // 120: vtctlservice.Vtctld.ApplyVSchema:output_type -> vtctldata.ApplyVSchemaResponse - 121, // 121: vtctlservice.Vtctld.Backup:output_type -> vtctldata.BackupResponse - 121, // 122: vtctlservice.Vtctld.BackupShard:output_type -> vtctldata.BackupResponse - 122, // 123: vtctlservice.Vtctld.CancelSchemaMigration:output_type -> vtctldata.CancelSchemaMigrationResponse - 123, // 124: vtctlservice.Vtctld.ChangeTabletType:output_type -> vtctldata.ChangeTabletTypeResponse - 124, // 125: vtctlservice.Vtctld.CleanupSchemaMigration:output_type -> vtctldata.CleanupSchemaMigrationResponse - 125, // 126: vtctlservice.Vtctld.CompleteSchemaMigration:output_type -> vtctldata.CompleteSchemaMigrationResponse - 126, // 127: vtctlservice.Vtctld.CreateKeyspace:output_type -> vtctldata.CreateKeyspaceResponse - 127, // 128: vtctlservice.Vtctld.CreateShard:output_type -> vtctldata.CreateShardResponse - 128, // 129: vtctlservice.Vtctld.DeleteCellInfo:output_type -> vtctldata.DeleteCellInfoResponse - 129, // 130: vtctlservice.Vtctld.DeleteCellsAlias:output_type -> vtctldata.DeleteCellsAliasResponse - 130, // 131: vtctlservice.Vtctld.DeleteKeyspace:output_type -> vtctldata.DeleteKeyspaceResponse - 131, // 132: vtctlservice.Vtctld.DeleteShards:output_type -> vtctldata.DeleteShardsResponse - 132, // 133: vtctlservice.Vtctld.DeleteSrvVSchema:output_type -> vtctldata.DeleteSrvVSchemaResponse - 133, // 134: vtctlservice.Vtctld.DeleteTablets:output_type -> vtctldata.DeleteTabletsResponse - 134, // 135: vtctlservice.Vtctld.EmergencyReparentShard:output_type -> vtctldata.EmergencyReparentShardResponse - 135, // 136: vtctlservice.Vtctld.ExecuteFetchAsApp:output_type -> vtctldata.ExecuteFetchAsAppResponse - 136, // 137: vtctlservice.Vtctld.ExecuteFetchAsDBA:output_type -> vtctldata.ExecuteFetchAsDBAResponse - 137, // 138: vtctlservice.Vtctld.ExecuteHook:output_type -> vtctldata.ExecuteHookResponse - 138, // 139: vtctlservice.Vtctld.ExecuteMultiFetchAsDBA:output_type -> vtctldata.ExecuteMultiFetchAsDBAResponse - 139, // 140: vtctlservice.Vtctld.FindAllShardsInKeyspace:output_type -> vtctldata.FindAllShardsInKeyspaceResponse - 140, // 141: vtctlservice.Vtctld.ForceCutOverSchemaMigration:output_type -> vtctldata.ForceCutOverSchemaMigrationResponse - 141, // 142: vtctlservice.Vtctld.GetBackups:output_type -> vtctldata.GetBackupsResponse - 142, // 143: vtctlservice.Vtctld.GetCellInfo:output_type -> vtctldata.GetCellInfoResponse - 143, // 144: vtctlservice.Vtctld.GetCellInfoNames:output_type -> vtctldata.GetCellInfoNamesResponse - 144, // 145: vtctlservice.Vtctld.GetCellsAliases:output_type -> vtctldata.GetCellsAliasesResponse - 145, // 146: vtctlservice.Vtctld.GetFullStatus:output_type -> vtctldata.GetFullStatusResponse - 146, // 147: vtctlservice.Vtctld.GetKeyspace:output_type -> vtctldata.GetKeyspaceResponse - 147, // 148: vtctlservice.Vtctld.GetKeyspaces:output_type -> vtctldata.GetKeyspacesResponse - 148, // 149: vtctlservice.Vtctld.GetPermissions:output_type -> vtctldata.GetPermissionsResponse - 149, // 150: vtctlservice.Vtctld.GetRoutingRules:output_type -> vtctldata.GetRoutingRulesResponse - 150, // 151: vtctlservice.Vtctld.GetSchema:output_type -> vtctldata.GetSchemaResponse - 151, // 152: vtctlservice.Vtctld.GetSchemaMigrations:output_type -> vtctldata.GetSchemaMigrationsResponse - 152, // 153: vtctlservice.Vtctld.GetShardReplication:output_type -> vtctldata.GetShardReplicationResponse - 153, // 154: vtctlservice.Vtctld.GetShard:output_type -> vtctldata.GetShardResponse - 154, // 155: vtctlservice.Vtctld.GetShardRoutingRules:output_type -> vtctldata.GetShardRoutingRulesResponse - 155, // 156: vtctlservice.Vtctld.GetSrvKeyspaceNames:output_type -> vtctldata.GetSrvKeyspaceNamesResponse - 156, // 157: vtctlservice.Vtctld.GetSrvKeyspaces:output_type -> vtctldata.GetSrvKeyspacesResponse - 157, // 158: vtctlservice.Vtctld.UpdateThrottlerConfig:output_type -> vtctldata.UpdateThrottlerConfigResponse - 158, // 159: vtctlservice.Vtctld.GetSrvVSchema:output_type -> vtctldata.GetSrvVSchemaResponse - 159, // 160: vtctlservice.Vtctld.GetSrvVSchemas:output_type -> vtctldata.GetSrvVSchemasResponse - 160, // 161: vtctlservice.Vtctld.GetTablet:output_type -> vtctldata.GetTabletResponse - 161, // 162: vtctlservice.Vtctld.GetTablets:output_type -> vtctldata.GetTabletsResponse - 162, // 163: vtctlservice.Vtctld.GetTopologyPath:output_type -> vtctldata.GetTopologyPathResponse - 163, // 164: vtctlservice.Vtctld.GetVersion:output_type -> vtctldata.GetVersionResponse - 164, // 165: vtctlservice.Vtctld.GetVSchema:output_type -> vtctldata.GetVSchemaResponse - 165, // 166: vtctlservice.Vtctld.GetWorkflows:output_type -> vtctldata.GetWorkflowsResponse - 166, // 167: vtctlservice.Vtctld.InitShardPrimary:output_type -> vtctldata.InitShardPrimaryResponse - 167, // 168: vtctlservice.Vtctld.LaunchSchemaMigration:output_type -> vtctldata.LaunchSchemaMigrationResponse - 168, // 169: vtctlservice.Vtctld.LookupVindexCreate:output_type -> vtctldata.LookupVindexCreateResponse - 169, // 170: vtctlservice.Vtctld.LookupVindexExternalize:output_type -> vtctldata.LookupVindexExternalizeResponse - 170, // 171: vtctlservice.Vtctld.MaterializeCreate:output_type -> vtctldata.MaterializeCreateResponse - 171, // 172: vtctlservice.Vtctld.MigrateCreate:output_type -> vtctldata.WorkflowStatusResponse - 172, // 173: vtctlservice.Vtctld.MountRegister:output_type -> vtctldata.MountRegisterResponse - 173, // 174: vtctlservice.Vtctld.MountUnregister:output_type -> vtctldata.MountUnregisterResponse - 174, // 175: vtctlservice.Vtctld.MountShow:output_type -> vtctldata.MountShowResponse - 175, // 176: vtctlservice.Vtctld.MountList:output_type -> vtctldata.MountListResponse - 171, // 177: vtctlservice.Vtctld.MoveTablesCreate:output_type -> vtctldata.WorkflowStatusResponse - 176, // 178: vtctlservice.Vtctld.MoveTablesComplete:output_type -> vtctldata.MoveTablesCompleteResponse - 177, // 179: vtctlservice.Vtctld.PingTablet:output_type -> vtctldata.PingTabletResponse - 178, // 180: vtctlservice.Vtctld.PlannedReparentShard:output_type -> vtctldata.PlannedReparentShardResponse - 179, // 181: vtctlservice.Vtctld.RebuildKeyspaceGraph:output_type -> vtctldata.RebuildKeyspaceGraphResponse - 180, // 182: vtctlservice.Vtctld.RebuildVSchemaGraph:output_type -> vtctldata.RebuildVSchemaGraphResponse - 181, // 183: vtctlservice.Vtctld.RefreshState:output_type -> vtctldata.RefreshStateResponse - 182, // 184: vtctlservice.Vtctld.RefreshStateByShard:output_type -> vtctldata.RefreshStateByShardResponse - 183, // 185: vtctlservice.Vtctld.ReloadSchema:output_type -> vtctldata.ReloadSchemaResponse - 184, // 186: vtctlservice.Vtctld.ReloadSchemaKeyspace:output_type -> vtctldata.ReloadSchemaKeyspaceResponse - 185, // 187: vtctlservice.Vtctld.ReloadSchemaShard:output_type -> vtctldata.ReloadSchemaShardResponse - 186, // 188: vtctlservice.Vtctld.RemoveBackup:output_type -> vtctldata.RemoveBackupResponse - 187, // 189: vtctlservice.Vtctld.RemoveKeyspaceCell:output_type -> vtctldata.RemoveKeyspaceCellResponse - 188, // 190: vtctlservice.Vtctld.RemoveShardCell:output_type -> vtctldata.RemoveShardCellResponse - 189, // 191: vtctlservice.Vtctld.ReparentTablet:output_type -> vtctldata.ReparentTabletResponse - 171, // 192: vtctlservice.Vtctld.ReshardCreate:output_type -> vtctldata.WorkflowStatusResponse - 190, // 193: vtctlservice.Vtctld.RestoreFromBackup:output_type -> vtctldata.RestoreFromBackupResponse - 191, // 194: vtctlservice.Vtctld.RetrySchemaMigration:output_type -> vtctldata.RetrySchemaMigrationResponse - 192, // 195: vtctlservice.Vtctld.RunHealthCheck:output_type -> vtctldata.RunHealthCheckResponse - 193, // 196: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:output_type -> vtctldata.SetKeyspaceDurabilityPolicyResponse - 194, // 197: vtctlservice.Vtctld.SetShardIsPrimaryServing:output_type -> vtctldata.SetShardIsPrimaryServingResponse - 195, // 198: vtctlservice.Vtctld.SetShardTabletControl:output_type -> vtctldata.SetShardTabletControlResponse - 196, // 199: vtctlservice.Vtctld.SetWritable:output_type -> vtctldata.SetWritableResponse - 197, // 200: vtctlservice.Vtctld.ShardReplicationAdd:output_type -> vtctldata.ShardReplicationAddResponse - 198, // 201: vtctlservice.Vtctld.ShardReplicationFix:output_type -> vtctldata.ShardReplicationFixResponse - 199, // 202: vtctlservice.Vtctld.ShardReplicationPositions:output_type -> vtctldata.ShardReplicationPositionsResponse - 200, // 203: vtctlservice.Vtctld.ShardReplicationRemove:output_type -> vtctldata.ShardReplicationRemoveResponse - 201, // 204: vtctlservice.Vtctld.SleepTablet:output_type -> vtctldata.SleepTabletResponse - 202, // 205: vtctlservice.Vtctld.SourceShardAdd:output_type -> vtctldata.SourceShardAddResponse - 203, // 206: vtctlservice.Vtctld.SourceShardDelete:output_type -> vtctldata.SourceShardDeleteResponse - 204, // 207: vtctlservice.Vtctld.StartReplication:output_type -> vtctldata.StartReplicationResponse - 205, // 208: vtctlservice.Vtctld.StopReplication:output_type -> vtctldata.StopReplicationResponse - 206, // 209: vtctlservice.Vtctld.TabletExternallyReparented:output_type -> vtctldata.TabletExternallyReparentedResponse - 207, // 210: vtctlservice.Vtctld.UpdateCellInfo:output_type -> vtctldata.UpdateCellInfoResponse - 208, // 211: vtctlservice.Vtctld.UpdateCellsAlias:output_type -> vtctldata.UpdateCellsAliasResponse - 209, // 212: vtctlservice.Vtctld.Validate:output_type -> vtctldata.ValidateResponse - 210, // 213: vtctlservice.Vtctld.ValidateKeyspace:output_type -> vtctldata.ValidateKeyspaceResponse - 211, // 214: vtctlservice.Vtctld.ValidateSchemaKeyspace:output_type -> vtctldata.ValidateSchemaKeyspaceResponse - 212, // 215: vtctlservice.Vtctld.ValidateShard:output_type -> vtctldata.ValidateShardResponse - 213, // 216: vtctlservice.Vtctld.ValidateVersionKeyspace:output_type -> vtctldata.ValidateVersionKeyspaceResponse - 214, // 217: vtctlservice.Vtctld.ValidateVersionShard:output_type -> vtctldata.ValidateVersionShardResponse - 215, // 218: vtctlservice.Vtctld.ValidateVSchema:output_type -> vtctldata.ValidateVSchemaResponse - 216, // 219: vtctlservice.Vtctld.VDiffCreate:output_type -> vtctldata.VDiffCreateResponse - 217, // 220: vtctlservice.Vtctld.VDiffDelete:output_type -> vtctldata.VDiffDeleteResponse - 218, // 221: vtctlservice.Vtctld.VDiffResume:output_type -> vtctldata.VDiffResumeResponse - 219, // 222: vtctlservice.Vtctld.VDiffShow:output_type -> vtctldata.VDiffShowResponse - 220, // 223: vtctlservice.Vtctld.VDiffStop:output_type -> vtctldata.VDiffStopResponse - 221, // 224: vtctlservice.Vtctld.WorkflowDelete:output_type -> vtctldata.WorkflowDeleteResponse - 171, // 225: vtctlservice.Vtctld.WorkflowStatus:output_type -> vtctldata.WorkflowStatusResponse - 222, // 226: vtctlservice.Vtctld.WorkflowSwitchTraffic:output_type -> vtctldata.WorkflowSwitchTrafficResponse - 223, // 227: vtctlservice.Vtctld.WorkflowUpdate:output_type -> vtctldata.WorkflowUpdateResponse - 114, // [114:228] is the sub-list for method output_type - 0, // [0:114] is the sub-list for method input_type + 5, // 5: vtctlservice.Vtctld.ApplyKeyspaceRoutingRules:input_type -> vtctldata.ApplyKeyspaceRoutingRulesRequest + 6, // 6: vtctlservice.Vtctld.ApplyShardRoutingRules:input_type -> vtctldata.ApplyShardRoutingRulesRequest + 7, // 7: vtctlservice.Vtctld.ApplyVSchema:input_type -> vtctldata.ApplyVSchemaRequest + 8, // 8: vtctlservice.Vtctld.Backup:input_type -> vtctldata.BackupRequest + 9, // 9: vtctlservice.Vtctld.BackupShard:input_type -> vtctldata.BackupShardRequest + 10, // 10: vtctlservice.Vtctld.CancelSchemaMigration:input_type -> vtctldata.CancelSchemaMigrationRequest + 11, // 11: vtctlservice.Vtctld.ChangeTabletType:input_type -> vtctldata.ChangeTabletTypeRequest + 12, // 12: vtctlservice.Vtctld.CleanupSchemaMigration:input_type -> vtctldata.CleanupSchemaMigrationRequest + 13, // 13: vtctlservice.Vtctld.CompleteSchemaMigration:input_type -> vtctldata.CompleteSchemaMigrationRequest + 14, // 14: vtctlservice.Vtctld.CreateKeyspace:input_type -> vtctldata.CreateKeyspaceRequest + 15, // 15: vtctlservice.Vtctld.CreateShard:input_type -> vtctldata.CreateShardRequest + 16, // 16: vtctlservice.Vtctld.DeleteCellInfo:input_type -> vtctldata.DeleteCellInfoRequest + 17, // 17: vtctlservice.Vtctld.DeleteCellsAlias:input_type -> vtctldata.DeleteCellsAliasRequest + 18, // 18: vtctlservice.Vtctld.DeleteKeyspace:input_type -> vtctldata.DeleteKeyspaceRequest + 19, // 19: vtctlservice.Vtctld.DeleteShards:input_type -> vtctldata.DeleteShardsRequest + 20, // 20: vtctlservice.Vtctld.DeleteSrvVSchema:input_type -> vtctldata.DeleteSrvVSchemaRequest + 21, // 21: vtctlservice.Vtctld.DeleteTablets:input_type -> vtctldata.DeleteTabletsRequest + 22, // 22: vtctlservice.Vtctld.EmergencyReparentShard:input_type -> vtctldata.EmergencyReparentShardRequest + 23, // 23: vtctlservice.Vtctld.ExecuteFetchAsApp:input_type -> vtctldata.ExecuteFetchAsAppRequest + 24, // 24: vtctlservice.Vtctld.ExecuteFetchAsDBA:input_type -> vtctldata.ExecuteFetchAsDBARequest + 25, // 25: vtctlservice.Vtctld.ExecuteHook:input_type -> vtctldata.ExecuteHookRequest + 26, // 26: vtctlservice.Vtctld.ExecuteMultiFetchAsDBA:input_type -> vtctldata.ExecuteMultiFetchAsDBARequest + 27, // 27: vtctlservice.Vtctld.FindAllShardsInKeyspace:input_type -> vtctldata.FindAllShardsInKeyspaceRequest + 28, // 28: vtctlservice.Vtctld.ForceCutOverSchemaMigration:input_type -> vtctldata.ForceCutOverSchemaMigrationRequest + 29, // 29: vtctlservice.Vtctld.GetBackups:input_type -> vtctldata.GetBackupsRequest + 30, // 30: vtctlservice.Vtctld.GetCellInfo:input_type -> vtctldata.GetCellInfoRequest + 31, // 31: vtctlservice.Vtctld.GetCellInfoNames:input_type -> vtctldata.GetCellInfoNamesRequest + 32, // 32: vtctlservice.Vtctld.GetCellsAliases:input_type -> vtctldata.GetCellsAliasesRequest + 33, // 33: vtctlservice.Vtctld.GetFullStatus:input_type -> vtctldata.GetFullStatusRequest + 34, // 34: vtctlservice.Vtctld.GetKeyspace:input_type -> vtctldata.GetKeyspaceRequest + 35, // 35: vtctlservice.Vtctld.GetKeyspaces:input_type -> vtctldata.GetKeyspacesRequest + 36, // 36: vtctlservice.Vtctld.GetKeyspaceRoutingRules:input_type -> vtctldata.GetKeyspaceRoutingRulesRequest + 37, // 37: vtctlservice.Vtctld.GetPermissions:input_type -> vtctldata.GetPermissionsRequest + 38, // 38: vtctlservice.Vtctld.GetRoutingRules:input_type -> vtctldata.GetRoutingRulesRequest + 39, // 39: vtctlservice.Vtctld.GetSchema:input_type -> vtctldata.GetSchemaRequest + 40, // 40: vtctlservice.Vtctld.GetSchemaMigrations:input_type -> vtctldata.GetSchemaMigrationsRequest + 41, // 41: vtctlservice.Vtctld.GetShardReplication:input_type -> vtctldata.GetShardReplicationRequest + 42, // 42: vtctlservice.Vtctld.GetShard:input_type -> vtctldata.GetShardRequest + 43, // 43: vtctlservice.Vtctld.GetShardRoutingRules:input_type -> vtctldata.GetShardRoutingRulesRequest + 44, // 44: vtctlservice.Vtctld.GetSrvKeyspaceNames:input_type -> vtctldata.GetSrvKeyspaceNamesRequest + 45, // 45: vtctlservice.Vtctld.GetSrvKeyspaces:input_type -> vtctldata.GetSrvKeyspacesRequest + 46, // 46: vtctlservice.Vtctld.UpdateThrottlerConfig:input_type -> vtctldata.UpdateThrottlerConfigRequest + 47, // 47: vtctlservice.Vtctld.GetSrvVSchema:input_type -> vtctldata.GetSrvVSchemaRequest + 48, // 48: vtctlservice.Vtctld.GetSrvVSchemas:input_type -> vtctldata.GetSrvVSchemasRequest + 49, // 49: vtctlservice.Vtctld.GetTablet:input_type -> vtctldata.GetTabletRequest + 50, // 50: vtctlservice.Vtctld.GetTablets:input_type -> vtctldata.GetTabletsRequest + 51, // 51: vtctlservice.Vtctld.GetTopologyPath:input_type -> vtctldata.GetTopologyPathRequest + 52, // 52: vtctlservice.Vtctld.GetVersion:input_type -> vtctldata.GetVersionRequest + 53, // 53: vtctlservice.Vtctld.GetVSchema:input_type -> vtctldata.GetVSchemaRequest + 54, // 54: vtctlservice.Vtctld.GetWorkflows:input_type -> vtctldata.GetWorkflowsRequest + 55, // 55: vtctlservice.Vtctld.InitShardPrimary:input_type -> vtctldata.InitShardPrimaryRequest + 56, // 56: vtctlservice.Vtctld.LaunchSchemaMigration:input_type -> vtctldata.LaunchSchemaMigrationRequest + 57, // 57: vtctlservice.Vtctld.LookupVindexCreate:input_type -> vtctldata.LookupVindexCreateRequest + 58, // 58: vtctlservice.Vtctld.LookupVindexExternalize:input_type -> vtctldata.LookupVindexExternalizeRequest + 59, // 59: vtctlservice.Vtctld.MaterializeCreate:input_type -> vtctldata.MaterializeCreateRequest + 60, // 60: vtctlservice.Vtctld.MigrateCreate:input_type -> vtctldata.MigrateCreateRequest + 61, // 61: vtctlservice.Vtctld.MountRegister:input_type -> vtctldata.MountRegisterRequest + 62, // 62: vtctlservice.Vtctld.MountUnregister:input_type -> vtctldata.MountUnregisterRequest + 63, // 63: vtctlservice.Vtctld.MountShow:input_type -> vtctldata.MountShowRequest + 64, // 64: vtctlservice.Vtctld.MountList:input_type -> vtctldata.MountListRequest + 65, // 65: vtctlservice.Vtctld.MoveTablesCreate:input_type -> vtctldata.MoveTablesCreateRequest + 66, // 66: vtctlservice.Vtctld.MoveTablesComplete:input_type -> vtctldata.MoveTablesCompleteRequest + 67, // 67: vtctlservice.Vtctld.PingTablet:input_type -> vtctldata.PingTabletRequest + 68, // 68: vtctlservice.Vtctld.PlannedReparentShard:input_type -> vtctldata.PlannedReparentShardRequest + 69, // 69: vtctlservice.Vtctld.RebuildKeyspaceGraph:input_type -> vtctldata.RebuildKeyspaceGraphRequest + 70, // 70: vtctlservice.Vtctld.RebuildVSchemaGraph:input_type -> vtctldata.RebuildVSchemaGraphRequest + 71, // 71: vtctlservice.Vtctld.RefreshState:input_type -> vtctldata.RefreshStateRequest + 72, // 72: vtctlservice.Vtctld.RefreshStateByShard:input_type -> vtctldata.RefreshStateByShardRequest + 73, // 73: vtctlservice.Vtctld.ReloadSchema:input_type -> vtctldata.ReloadSchemaRequest + 74, // 74: vtctlservice.Vtctld.ReloadSchemaKeyspace:input_type -> vtctldata.ReloadSchemaKeyspaceRequest + 75, // 75: vtctlservice.Vtctld.ReloadSchemaShard:input_type -> vtctldata.ReloadSchemaShardRequest + 76, // 76: vtctlservice.Vtctld.RemoveBackup:input_type -> vtctldata.RemoveBackupRequest + 77, // 77: vtctlservice.Vtctld.RemoveKeyspaceCell:input_type -> vtctldata.RemoveKeyspaceCellRequest + 78, // 78: vtctlservice.Vtctld.RemoveShardCell:input_type -> vtctldata.RemoveShardCellRequest + 79, // 79: vtctlservice.Vtctld.ReparentTablet:input_type -> vtctldata.ReparentTabletRequest + 80, // 80: vtctlservice.Vtctld.ReshardCreate:input_type -> vtctldata.ReshardCreateRequest + 81, // 81: vtctlservice.Vtctld.RestoreFromBackup:input_type -> vtctldata.RestoreFromBackupRequest + 82, // 82: vtctlservice.Vtctld.RetrySchemaMigration:input_type -> vtctldata.RetrySchemaMigrationRequest + 83, // 83: vtctlservice.Vtctld.RunHealthCheck:input_type -> vtctldata.RunHealthCheckRequest + 84, // 84: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:input_type -> vtctldata.SetKeyspaceDurabilityPolicyRequest + 85, // 85: vtctlservice.Vtctld.SetShardIsPrimaryServing:input_type -> vtctldata.SetShardIsPrimaryServingRequest + 86, // 86: vtctlservice.Vtctld.SetShardTabletControl:input_type -> vtctldata.SetShardTabletControlRequest + 87, // 87: vtctlservice.Vtctld.SetWritable:input_type -> vtctldata.SetWritableRequest + 88, // 88: vtctlservice.Vtctld.ShardReplicationAdd:input_type -> vtctldata.ShardReplicationAddRequest + 89, // 89: vtctlservice.Vtctld.ShardReplicationFix:input_type -> vtctldata.ShardReplicationFixRequest + 90, // 90: vtctlservice.Vtctld.ShardReplicationPositions:input_type -> vtctldata.ShardReplicationPositionsRequest + 91, // 91: vtctlservice.Vtctld.ShardReplicationRemove:input_type -> vtctldata.ShardReplicationRemoveRequest + 92, // 92: vtctlservice.Vtctld.SleepTablet:input_type -> vtctldata.SleepTabletRequest + 93, // 93: vtctlservice.Vtctld.SourceShardAdd:input_type -> vtctldata.SourceShardAddRequest + 94, // 94: vtctlservice.Vtctld.SourceShardDelete:input_type -> vtctldata.SourceShardDeleteRequest + 95, // 95: vtctlservice.Vtctld.StartReplication:input_type -> vtctldata.StartReplicationRequest + 96, // 96: vtctlservice.Vtctld.StopReplication:input_type -> vtctldata.StopReplicationRequest + 97, // 97: vtctlservice.Vtctld.TabletExternallyReparented:input_type -> vtctldata.TabletExternallyReparentedRequest + 98, // 98: vtctlservice.Vtctld.UpdateCellInfo:input_type -> vtctldata.UpdateCellInfoRequest + 99, // 99: vtctlservice.Vtctld.UpdateCellsAlias:input_type -> vtctldata.UpdateCellsAliasRequest + 100, // 100: vtctlservice.Vtctld.Validate:input_type -> vtctldata.ValidateRequest + 101, // 101: vtctlservice.Vtctld.ValidateKeyspace:input_type -> vtctldata.ValidateKeyspaceRequest + 102, // 102: vtctlservice.Vtctld.ValidateSchemaKeyspace:input_type -> vtctldata.ValidateSchemaKeyspaceRequest + 103, // 103: vtctlservice.Vtctld.ValidateShard:input_type -> vtctldata.ValidateShardRequest + 104, // 104: vtctlservice.Vtctld.ValidateVersionKeyspace:input_type -> vtctldata.ValidateVersionKeyspaceRequest + 105, // 105: vtctlservice.Vtctld.ValidateVersionShard:input_type -> vtctldata.ValidateVersionShardRequest + 106, // 106: vtctlservice.Vtctld.ValidateVSchema:input_type -> vtctldata.ValidateVSchemaRequest + 107, // 107: vtctlservice.Vtctld.VDiffCreate:input_type -> vtctldata.VDiffCreateRequest + 108, // 108: vtctlservice.Vtctld.VDiffDelete:input_type -> vtctldata.VDiffDeleteRequest + 109, // 109: vtctlservice.Vtctld.VDiffResume:input_type -> vtctldata.VDiffResumeRequest + 110, // 110: vtctlservice.Vtctld.VDiffShow:input_type -> vtctldata.VDiffShowRequest + 111, // 111: vtctlservice.Vtctld.VDiffStop:input_type -> vtctldata.VDiffStopRequest + 112, // 112: vtctlservice.Vtctld.WorkflowDelete:input_type -> vtctldata.WorkflowDeleteRequest + 113, // 113: vtctlservice.Vtctld.WorkflowStatus:input_type -> vtctldata.WorkflowStatusRequest + 114, // 114: vtctlservice.Vtctld.WorkflowSwitchTraffic:input_type -> vtctldata.WorkflowSwitchTrafficRequest + 115, // 115: vtctlservice.Vtctld.WorkflowUpdate:input_type -> vtctldata.WorkflowUpdateRequest + 116, // 116: vtctlservice.Vtctl.ExecuteVtctlCommand:output_type -> vtctldata.ExecuteVtctlCommandResponse + 117, // 117: vtctlservice.Vtctld.AddCellInfo:output_type -> vtctldata.AddCellInfoResponse + 118, // 118: vtctlservice.Vtctld.AddCellsAlias:output_type -> vtctldata.AddCellsAliasResponse + 119, // 119: vtctlservice.Vtctld.ApplyRoutingRules:output_type -> vtctldata.ApplyRoutingRulesResponse + 120, // 120: vtctlservice.Vtctld.ApplySchema:output_type -> vtctldata.ApplySchemaResponse + 121, // 121: vtctlservice.Vtctld.ApplyKeyspaceRoutingRules:output_type -> vtctldata.ApplyKeyspaceRoutingRulesResponse + 122, // 122: vtctlservice.Vtctld.ApplyShardRoutingRules:output_type -> vtctldata.ApplyShardRoutingRulesResponse + 123, // 123: vtctlservice.Vtctld.ApplyVSchema:output_type -> vtctldata.ApplyVSchemaResponse + 124, // 124: vtctlservice.Vtctld.Backup:output_type -> vtctldata.BackupResponse + 124, // 125: vtctlservice.Vtctld.BackupShard:output_type -> vtctldata.BackupResponse + 125, // 126: vtctlservice.Vtctld.CancelSchemaMigration:output_type -> vtctldata.CancelSchemaMigrationResponse + 126, // 127: vtctlservice.Vtctld.ChangeTabletType:output_type -> vtctldata.ChangeTabletTypeResponse + 127, // 128: vtctlservice.Vtctld.CleanupSchemaMigration:output_type -> vtctldata.CleanupSchemaMigrationResponse + 128, // 129: vtctlservice.Vtctld.CompleteSchemaMigration:output_type -> vtctldata.CompleteSchemaMigrationResponse + 129, // 130: vtctlservice.Vtctld.CreateKeyspace:output_type -> vtctldata.CreateKeyspaceResponse + 130, // 131: vtctlservice.Vtctld.CreateShard:output_type -> vtctldata.CreateShardResponse + 131, // 132: vtctlservice.Vtctld.DeleteCellInfo:output_type -> vtctldata.DeleteCellInfoResponse + 132, // 133: vtctlservice.Vtctld.DeleteCellsAlias:output_type -> vtctldata.DeleteCellsAliasResponse + 133, // 134: vtctlservice.Vtctld.DeleteKeyspace:output_type -> vtctldata.DeleteKeyspaceResponse + 134, // 135: vtctlservice.Vtctld.DeleteShards:output_type -> vtctldata.DeleteShardsResponse + 135, // 136: vtctlservice.Vtctld.DeleteSrvVSchema:output_type -> vtctldata.DeleteSrvVSchemaResponse + 136, // 137: vtctlservice.Vtctld.DeleteTablets:output_type -> vtctldata.DeleteTabletsResponse + 137, // 138: vtctlservice.Vtctld.EmergencyReparentShard:output_type -> vtctldata.EmergencyReparentShardResponse + 138, // 139: vtctlservice.Vtctld.ExecuteFetchAsApp:output_type -> vtctldata.ExecuteFetchAsAppResponse + 139, // 140: vtctlservice.Vtctld.ExecuteFetchAsDBA:output_type -> vtctldata.ExecuteFetchAsDBAResponse + 140, // 141: vtctlservice.Vtctld.ExecuteHook:output_type -> vtctldata.ExecuteHookResponse + 141, // 142: vtctlservice.Vtctld.ExecuteMultiFetchAsDBA:output_type -> vtctldata.ExecuteMultiFetchAsDBAResponse + 142, // 143: vtctlservice.Vtctld.FindAllShardsInKeyspace:output_type -> vtctldata.FindAllShardsInKeyspaceResponse + 143, // 144: vtctlservice.Vtctld.ForceCutOverSchemaMigration:output_type -> vtctldata.ForceCutOverSchemaMigrationResponse + 144, // 145: vtctlservice.Vtctld.GetBackups:output_type -> vtctldata.GetBackupsResponse + 145, // 146: vtctlservice.Vtctld.GetCellInfo:output_type -> vtctldata.GetCellInfoResponse + 146, // 147: vtctlservice.Vtctld.GetCellInfoNames:output_type -> vtctldata.GetCellInfoNamesResponse + 147, // 148: vtctlservice.Vtctld.GetCellsAliases:output_type -> vtctldata.GetCellsAliasesResponse + 148, // 149: vtctlservice.Vtctld.GetFullStatus:output_type -> vtctldata.GetFullStatusResponse + 149, // 150: vtctlservice.Vtctld.GetKeyspace:output_type -> vtctldata.GetKeyspaceResponse + 150, // 151: vtctlservice.Vtctld.GetKeyspaces:output_type -> vtctldata.GetKeyspacesResponse + 151, // 152: vtctlservice.Vtctld.GetKeyspaceRoutingRules:output_type -> vtctldata.GetKeyspaceRoutingRulesResponse + 152, // 153: vtctlservice.Vtctld.GetPermissions:output_type -> vtctldata.GetPermissionsResponse + 153, // 154: vtctlservice.Vtctld.GetRoutingRules:output_type -> vtctldata.GetRoutingRulesResponse + 154, // 155: vtctlservice.Vtctld.GetSchema:output_type -> vtctldata.GetSchemaResponse + 155, // 156: vtctlservice.Vtctld.GetSchemaMigrations:output_type -> vtctldata.GetSchemaMigrationsResponse + 156, // 157: vtctlservice.Vtctld.GetShardReplication:output_type -> vtctldata.GetShardReplicationResponse + 157, // 158: vtctlservice.Vtctld.GetShard:output_type -> vtctldata.GetShardResponse + 158, // 159: vtctlservice.Vtctld.GetShardRoutingRules:output_type -> vtctldata.GetShardRoutingRulesResponse + 159, // 160: vtctlservice.Vtctld.GetSrvKeyspaceNames:output_type -> vtctldata.GetSrvKeyspaceNamesResponse + 160, // 161: vtctlservice.Vtctld.GetSrvKeyspaces:output_type -> vtctldata.GetSrvKeyspacesResponse + 161, // 162: vtctlservice.Vtctld.UpdateThrottlerConfig:output_type -> vtctldata.UpdateThrottlerConfigResponse + 162, // 163: vtctlservice.Vtctld.GetSrvVSchema:output_type -> vtctldata.GetSrvVSchemaResponse + 163, // 164: vtctlservice.Vtctld.GetSrvVSchemas:output_type -> vtctldata.GetSrvVSchemasResponse + 164, // 165: vtctlservice.Vtctld.GetTablet:output_type -> vtctldata.GetTabletResponse + 165, // 166: vtctlservice.Vtctld.GetTablets:output_type -> vtctldata.GetTabletsResponse + 166, // 167: vtctlservice.Vtctld.GetTopologyPath:output_type -> vtctldata.GetTopologyPathResponse + 167, // 168: vtctlservice.Vtctld.GetVersion:output_type -> vtctldata.GetVersionResponse + 168, // 169: vtctlservice.Vtctld.GetVSchema:output_type -> vtctldata.GetVSchemaResponse + 169, // 170: vtctlservice.Vtctld.GetWorkflows:output_type -> vtctldata.GetWorkflowsResponse + 170, // 171: vtctlservice.Vtctld.InitShardPrimary:output_type -> vtctldata.InitShardPrimaryResponse + 171, // 172: vtctlservice.Vtctld.LaunchSchemaMigration:output_type -> vtctldata.LaunchSchemaMigrationResponse + 172, // 173: vtctlservice.Vtctld.LookupVindexCreate:output_type -> vtctldata.LookupVindexCreateResponse + 173, // 174: vtctlservice.Vtctld.LookupVindexExternalize:output_type -> vtctldata.LookupVindexExternalizeResponse + 174, // 175: vtctlservice.Vtctld.MaterializeCreate:output_type -> vtctldata.MaterializeCreateResponse + 175, // 176: vtctlservice.Vtctld.MigrateCreate:output_type -> vtctldata.WorkflowStatusResponse + 176, // 177: vtctlservice.Vtctld.MountRegister:output_type -> vtctldata.MountRegisterResponse + 177, // 178: vtctlservice.Vtctld.MountUnregister:output_type -> vtctldata.MountUnregisterResponse + 178, // 179: vtctlservice.Vtctld.MountShow:output_type -> vtctldata.MountShowResponse + 179, // 180: vtctlservice.Vtctld.MountList:output_type -> vtctldata.MountListResponse + 175, // 181: vtctlservice.Vtctld.MoveTablesCreate:output_type -> vtctldata.WorkflowStatusResponse + 180, // 182: vtctlservice.Vtctld.MoveTablesComplete:output_type -> vtctldata.MoveTablesCompleteResponse + 181, // 183: vtctlservice.Vtctld.PingTablet:output_type -> vtctldata.PingTabletResponse + 182, // 184: vtctlservice.Vtctld.PlannedReparentShard:output_type -> vtctldata.PlannedReparentShardResponse + 183, // 185: vtctlservice.Vtctld.RebuildKeyspaceGraph:output_type -> vtctldata.RebuildKeyspaceGraphResponse + 184, // 186: vtctlservice.Vtctld.RebuildVSchemaGraph:output_type -> vtctldata.RebuildVSchemaGraphResponse + 185, // 187: vtctlservice.Vtctld.RefreshState:output_type -> vtctldata.RefreshStateResponse + 186, // 188: vtctlservice.Vtctld.RefreshStateByShard:output_type -> vtctldata.RefreshStateByShardResponse + 187, // 189: vtctlservice.Vtctld.ReloadSchema:output_type -> vtctldata.ReloadSchemaResponse + 188, // 190: vtctlservice.Vtctld.ReloadSchemaKeyspace:output_type -> vtctldata.ReloadSchemaKeyspaceResponse + 189, // 191: vtctlservice.Vtctld.ReloadSchemaShard:output_type -> vtctldata.ReloadSchemaShardResponse + 190, // 192: vtctlservice.Vtctld.RemoveBackup:output_type -> vtctldata.RemoveBackupResponse + 191, // 193: vtctlservice.Vtctld.RemoveKeyspaceCell:output_type -> vtctldata.RemoveKeyspaceCellResponse + 192, // 194: vtctlservice.Vtctld.RemoveShardCell:output_type -> vtctldata.RemoveShardCellResponse + 193, // 195: vtctlservice.Vtctld.ReparentTablet:output_type -> vtctldata.ReparentTabletResponse + 175, // 196: vtctlservice.Vtctld.ReshardCreate:output_type -> vtctldata.WorkflowStatusResponse + 194, // 197: vtctlservice.Vtctld.RestoreFromBackup:output_type -> vtctldata.RestoreFromBackupResponse + 195, // 198: vtctlservice.Vtctld.RetrySchemaMigration:output_type -> vtctldata.RetrySchemaMigrationResponse + 196, // 199: vtctlservice.Vtctld.RunHealthCheck:output_type -> vtctldata.RunHealthCheckResponse + 197, // 200: vtctlservice.Vtctld.SetKeyspaceDurabilityPolicy:output_type -> vtctldata.SetKeyspaceDurabilityPolicyResponse + 198, // 201: vtctlservice.Vtctld.SetShardIsPrimaryServing:output_type -> vtctldata.SetShardIsPrimaryServingResponse + 199, // 202: vtctlservice.Vtctld.SetShardTabletControl:output_type -> vtctldata.SetShardTabletControlResponse + 200, // 203: vtctlservice.Vtctld.SetWritable:output_type -> vtctldata.SetWritableResponse + 201, // 204: vtctlservice.Vtctld.ShardReplicationAdd:output_type -> vtctldata.ShardReplicationAddResponse + 202, // 205: vtctlservice.Vtctld.ShardReplicationFix:output_type -> vtctldata.ShardReplicationFixResponse + 203, // 206: vtctlservice.Vtctld.ShardReplicationPositions:output_type -> vtctldata.ShardReplicationPositionsResponse + 204, // 207: vtctlservice.Vtctld.ShardReplicationRemove:output_type -> vtctldata.ShardReplicationRemoveResponse + 205, // 208: vtctlservice.Vtctld.SleepTablet:output_type -> vtctldata.SleepTabletResponse + 206, // 209: vtctlservice.Vtctld.SourceShardAdd:output_type -> vtctldata.SourceShardAddResponse + 207, // 210: vtctlservice.Vtctld.SourceShardDelete:output_type -> vtctldata.SourceShardDeleteResponse + 208, // 211: vtctlservice.Vtctld.StartReplication:output_type -> vtctldata.StartReplicationResponse + 209, // 212: vtctlservice.Vtctld.StopReplication:output_type -> vtctldata.StopReplicationResponse + 210, // 213: vtctlservice.Vtctld.TabletExternallyReparented:output_type -> vtctldata.TabletExternallyReparentedResponse + 211, // 214: vtctlservice.Vtctld.UpdateCellInfo:output_type -> vtctldata.UpdateCellInfoResponse + 212, // 215: vtctlservice.Vtctld.UpdateCellsAlias:output_type -> vtctldata.UpdateCellsAliasResponse + 213, // 216: vtctlservice.Vtctld.Validate:output_type -> vtctldata.ValidateResponse + 214, // 217: vtctlservice.Vtctld.ValidateKeyspace:output_type -> vtctldata.ValidateKeyspaceResponse + 215, // 218: vtctlservice.Vtctld.ValidateSchemaKeyspace:output_type -> vtctldata.ValidateSchemaKeyspaceResponse + 216, // 219: vtctlservice.Vtctld.ValidateShard:output_type -> vtctldata.ValidateShardResponse + 217, // 220: vtctlservice.Vtctld.ValidateVersionKeyspace:output_type -> vtctldata.ValidateVersionKeyspaceResponse + 218, // 221: vtctlservice.Vtctld.ValidateVersionShard:output_type -> vtctldata.ValidateVersionShardResponse + 219, // 222: vtctlservice.Vtctld.ValidateVSchema:output_type -> vtctldata.ValidateVSchemaResponse + 220, // 223: vtctlservice.Vtctld.VDiffCreate:output_type -> vtctldata.VDiffCreateResponse + 221, // 224: vtctlservice.Vtctld.VDiffDelete:output_type -> vtctldata.VDiffDeleteResponse + 222, // 225: vtctlservice.Vtctld.VDiffResume:output_type -> vtctldata.VDiffResumeResponse + 223, // 226: vtctlservice.Vtctld.VDiffShow:output_type -> vtctldata.VDiffShowResponse + 224, // 227: vtctlservice.Vtctld.VDiffStop:output_type -> vtctldata.VDiffStopResponse + 225, // 228: vtctlservice.Vtctld.WorkflowDelete:output_type -> vtctldata.WorkflowDeleteResponse + 175, // 229: vtctlservice.Vtctld.WorkflowStatus:output_type -> vtctldata.WorkflowStatusResponse + 226, // 230: vtctlservice.Vtctld.WorkflowSwitchTraffic:output_type -> vtctldata.WorkflowSwitchTrafficResponse + 227, // 231: vtctlservice.Vtctld.WorkflowUpdate:output_type -> vtctldata.WorkflowUpdateResponse + 116, // [116:232] is the sub-list for method output_type + 0, // [0:116] 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/vtctlservice/vtctlservice_grpc.pb.go b/go/vt/proto/vtctlservice/vtctlservice_grpc.pb.go index b369a41b1f1..37448e9d850 100644 --- a/go/vt/proto/vtctlservice/vtctlservice_grpc.pb.go +++ b/go/vt/proto/vtctlservice/vtctlservice_grpc.pb.go @@ -150,6 +150,8 @@ type VtctldClient interface { ApplyRoutingRules(ctx context.Context, in *vtctldata.ApplyRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.ApplyRoutingRulesResponse, error) // ApplySchema applies a schema to a keyspace. ApplySchema(ctx context.Context, in *vtctldata.ApplySchemaRequest, opts ...grpc.CallOption) (*vtctldata.ApplySchemaResponse, error) + // ApplyKeyspaceRoutingRules applies the VSchema keyspace routing rules. + ApplyKeyspaceRoutingRules(ctx context.Context, in *vtctldata.ApplyKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.ApplyKeyspaceRoutingRulesResponse, error) // ApplyShardRoutingRules applies the VSchema shard routing rules. ApplyShardRoutingRules(ctx context.Context, in *vtctldata.ApplyShardRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.ApplyShardRoutingRulesResponse, error) // ApplyVSchema applies a vschema to a keyspace. @@ -228,6 +230,8 @@ type VtctldClient interface { GetKeyspace(ctx context.Context, in *vtctldata.GetKeyspaceRequest, opts ...grpc.CallOption) (*vtctldata.GetKeyspaceResponse, error) // GetKeyspaces returns the keyspace struct of all keyspaces in the topo. GetKeyspaces(ctx context.Context, in *vtctldata.GetKeyspacesRequest, opts ...grpc.CallOption) (*vtctldata.GetKeyspacesResponse, error) + // GetKeyspaceRoutingRules returns the VSchema keyspace routing rules. + GetKeyspaceRoutingRules(ctx context.Context, in *vtctldata.GetKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.GetKeyspaceRoutingRulesResponse, error) // GetPermissions returns the permissions set on the remote tablet. GetPermissions(ctx context.Context, in *vtctldata.GetPermissionsRequest, opts ...grpc.CallOption) (*vtctldata.GetPermissionsResponse, error) // GetRoutingRules returns the VSchema routing rules. @@ -502,6 +506,15 @@ func (c *vtctldClient) ApplySchema(ctx context.Context, in *vtctldata.ApplySchem return out, nil } +func (c *vtctldClient) ApplyKeyspaceRoutingRules(ctx context.Context, in *vtctldata.ApplyKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.ApplyKeyspaceRoutingRulesResponse, error) { + out := new(vtctldata.ApplyKeyspaceRoutingRulesResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/ApplyKeyspaceRoutingRules", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *vtctldClient) ApplyShardRoutingRules(ctx context.Context, in *vtctldata.ApplyShardRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.ApplyShardRoutingRulesResponse, error) { out := new(vtctldata.ApplyShardRoutingRulesResponse) err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/ApplyShardRoutingRules", in, out, opts...) @@ -818,6 +831,15 @@ func (c *vtctldClient) GetKeyspaces(ctx context.Context, in *vtctldata.GetKeyspa return out, nil } +func (c *vtctldClient) GetKeyspaceRoutingRules(ctx context.Context, in *vtctldata.GetKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldata.GetKeyspaceRoutingRulesResponse, error) { + out := new(vtctldata.GetKeyspaceRoutingRulesResponse) + err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/GetKeyspaceRoutingRules", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *vtctldClient) GetPermissions(ctx context.Context, in *vtctldata.GetPermissionsRequest, opts ...grpc.CallOption) (*vtctldata.GetPermissionsResponse, error) { out := new(vtctldata.GetPermissionsResponse) err := c.cc.Invoke(ctx, "/vtctlservice.Vtctld/GetPermissions", in, out, opts...) @@ -1570,6 +1592,8 @@ type VtctldServer interface { ApplyRoutingRules(context.Context, *vtctldata.ApplyRoutingRulesRequest) (*vtctldata.ApplyRoutingRulesResponse, error) // ApplySchema applies a schema to a keyspace. ApplySchema(context.Context, *vtctldata.ApplySchemaRequest) (*vtctldata.ApplySchemaResponse, error) + // ApplyKeyspaceRoutingRules applies the VSchema keyspace routing rules. + ApplyKeyspaceRoutingRules(context.Context, *vtctldata.ApplyKeyspaceRoutingRulesRequest) (*vtctldata.ApplyKeyspaceRoutingRulesResponse, error) // ApplyShardRoutingRules applies the VSchema shard routing rules. ApplyShardRoutingRules(context.Context, *vtctldata.ApplyShardRoutingRulesRequest) (*vtctldata.ApplyShardRoutingRulesResponse, error) // ApplyVSchema applies a vschema to a keyspace. @@ -1648,6 +1672,8 @@ type VtctldServer interface { GetKeyspace(context.Context, *vtctldata.GetKeyspaceRequest) (*vtctldata.GetKeyspaceResponse, error) // GetKeyspaces returns the keyspace struct of all keyspaces in the topo. GetKeyspaces(context.Context, *vtctldata.GetKeyspacesRequest) (*vtctldata.GetKeyspacesResponse, error) + // GetKeyspaceRoutingRules returns the VSchema keyspace routing rules. + GetKeyspaceRoutingRules(context.Context, *vtctldata.GetKeyspaceRoutingRulesRequest) (*vtctldata.GetKeyspaceRoutingRulesResponse, error) // GetPermissions returns the permissions set on the remote tablet. GetPermissions(context.Context, *vtctldata.GetPermissionsRequest) (*vtctldata.GetPermissionsResponse, error) // GetRoutingRules returns the VSchema routing rules. @@ -1895,6 +1921,9 @@ func (UnimplementedVtctldServer) ApplyRoutingRules(context.Context, *vtctldata.A func (UnimplementedVtctldServer) ApplySchema(context.Context, *vtctldata.ApplySchemaRequest) (*vtctldata.ApplySchemaResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ApplySchema not implemented") } +func (UnimplementedVtctldServer) ApplyKeyspaceRoutingRules(context.Context, *vtctldata.ApplyKeyspaceRoutingRulesRequest) (*vtctldata.ApplyKeyspaceRoutingRulesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ApplyKeyspaceRoutingRules not implemented") +} func (UnimplementedVtctldServer) ApplyShardRoutingRules(context.Context, *vtctldata.ApplyShardRoutingRulesRequest) (*vtctldata.ApplyShardRoutingRulesResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ApplyShardRoutingRules not implemented") } @@ -1985,6 +2014,9 @@ func (UnimplementedVtctldServer) GetKeyspace(context.Context, *vtctldata.GetKeys func (UnimplementedVtctldServer) GetKeyspaces(context.Context, *vtctldata.GetKeyspacesRequest) (*vtctldata.GetKeyspacesResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetKeyspaces not implemented") } +func (UnimplementedVtctldServer) GetKeyspaceRoutingRules(context.Context, *vtctldata.GetKeyspaceRoutingRulesRequest) (*vtctldata.GetKeyspaceRoutingRulesResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetKeyspaceRoutingRules not implemented") +} func (UnimplementedVtctldServer) GetPermissions(context.Context, *vtctldata.GetPermissionsRequest) (*vtctldata.GetPermissionsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetPermissions not implemented") } @@ -2307,6 +2339,24 @@ func _Vtctld_ApplySchema_Handler(srv interface{}, ctx context.Context, dec func( return interceptor(ctx, in, info, handler) } +func _Vtctld_ApplyKeyspaceRoutingRules_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.ApplyKeyspaceRoutingRulesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).ApplyKeyspaceRoutingRules(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/ApplyKeyspaceRoutingRules", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).ApplyKeyspaceRoutingRules(ctx, req.(*vtctldata.ApplyKeyspaceRoutingRulesRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Vtctld_ApplyShardRoutingRules_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(vtctldata.ApplyShardRoutingRulesRequest) if err := dec(in); err != nil { @@ -2853,6 +2903,24 @@ func _Vtctld_GetKeyspaces_Handler(srv interface{}, ctx context.Context, dec func return interceptor(ctx, in, info, handler) } +func _Vtctld_GetKeyspaceRoutingRules_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(vtctldata.GetKeyspaceRoutingRulesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(VtctldServer).GetKeyspaceRoutingRules(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/vtctlservice.Vtctld/GetKeyspaceRoutingRules", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(VtctldServer).GetKeyspaceRoutingRules(ctx, req.(*vtctldata.GetKeyspaceRoutingRulesRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Vtctld_GetPermissions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(vtctldata.GetPermissionsRequest) if err := dec(in); err != nil { @@ -4301,6 +4369,10 @@ var Vtctld_ServiceDesc = grpc.ServiceDesc{ MethodName: "ApplySchema", Handler: _Vtctld_ApplySchema_Handler, }, + { + MethodName: "ApplyKeyspaceRoutingRules", + Handler: _Vtctld_ApplyKeyspaceRoutingRules_Handler, + }, { MethodName: "ApplyShardRoutingRules", Handler: _Vtctld_ApplyShardRoutingRules_Handler, @@ -4413,6 +4485,10 @@ var Vtctld_ServiceDesc = grpc.ServiceDesc{ MethodName: "GetKeyspaces", Handler: _Vtctld_GetKeyspaces_Handler, }, + { + MethodName: "GetKeyspaceRoutingRules", + Handler: _Vtctld_GetKeyspaceRoutingRules_Handler, + }, { MethodName: "GetPermissions", Handler: _Vtctld_GetPermissions_Handler, diff --git a/go/vt/sidecardb/schema/vreplication/vreplication.sql b/go/vt/sidecardb/schema/vreplication/vreplication.sql index ce9badfd98f..6615cb5a2cc 100644 --- a/go/vt/sidecardb/schema/vreplication/vreplication.sql +++ b/go/vt/sidecardb/schema/vreplication/vreplication.sql @@ -38,6 +38,14 @@ CREATE TABLE IF NOT EXISTS vreplication `component_throttled` varchar(255) NOT NULL DEFAULT '', `workflow_sub_type` int NOT NULL DEFAULT '0', `defer_secondary_keys` tinyint(1) NOT NULL DEFAULT '0', + /* + The options column is used to store any applicable additional attributes for a vreplication workflow. + Current attributes (all optional) are: + - `tenant-id`: used to specify the tenant id for a multi-tenant migration. (MoveTables only) + - `source_keyspace_alias`: used in conjunction with `use_keyspace_routing_rules` to specify the source keyspace + alias which can be used in multi-tenant migrations. (optional) (MoveTables only) + */ + `options` json NOT NULL, PRIMARY KEY (`id`), KEY `workflow_idx` (`workflow`(64)) ) ENGINE = InnoDB diff --git a/go/vt/topo/server.go b/go/vt/topo/server.go index 1995e8b6ec4..d076bf1417c 100644 --- a/go/vt/topo/server.go +++ b/go/vt/topo/server.go @@ -67,18 +67,19 @@ const ( // Filenames for all object types. const ( - CellInfoFile = "CellInfo" - CellsAliasFile = "CellsAlias" - KeyspaceFile = "Keyspace" - ShardFile = "Shard" - VSchemaFile = "VSchema" - ShardReplicationFile = "ShardReplication" - TabletFile = "Tablet" - SrvVSchemaFile = "SrvVSchema" - SrvKeyspaceFile = "SrvKeyspace" - RoutingRulesFile = "RoutingRules" - ExternalClustersFile = "ExternalClusters" - ShardRoutingRulesFile = "ShardRoutingRules" + CellInfoFile = "CellInfo" + CellsAliasFile = "CellsAlias" + KeyspaceFile = "Keyspace" + ShardFile = "Shard" + VSchemaFile = "VSchema" + ShardReplicationFile = "ShardReplication" + TabletFile = "Tablet" + SrvVSchemaFile = "SrvVSchema" + SrvKeyspaceFile = "SrvKeyspace" + RoutingRulesFile = "RoutingRules" + ExternalClustersFile = "ExternalClusters" + ShardRoutingRulesFile = "ShardRoutingRules" + KeyspaceRoutingRulesFile = "KeyspaceRoutingRules" ) // Path for all object types. diff --git a/go/vt/topo/srv_vschema.go b/go/vt/topo/srv_vschema.go index 453e8c5f8d0..c118253e8a8 100644 --- a/go/vt/topo/srv_vschema.go +++ b/go/vt/topo/srv_vschema.go @@ -204,6 +204,12 @@ func (ts *Server) RebuildSrvVSchema(ctx context.Context, cells []string) error { } srvVSchema.ShardRoutingRules = srr + krr, err := ts.GetKeyspaceRoutingRules(ctx) + if err != nil { + return fmt.Errorf("GetKeyspaceRoutingRules failed: %v", err) + } + srvVSchema.KeyspaceRoutingRules = krr + // now save the SrvVSchema in all cells in parallel for _, cell := range cells { wg.Add(1) diff --git a/go/vt/topo/vschema.go b/go/vt/topo/vschema.go index 0f63a26c2ae..b5290310d7d 100644 --- a/go/vt/topo/vschema.go +++ b/go/vt/topo/vschema.go @@ -158,3 +158,35 @@ func (ts *Server) GetShardRoutingRules(ctx context.Context) (*vschemapb.ShardRou } return srr, nil } + +func (ts *Server) SaveKeyspaceRoutingRules(ctx context.Context, rules *vschemapb.KeyspaceRoutingRules) error { + data, err := rules.MarshalVT() + if err != nil { + return err + } + if len(data) == 0 { + // No rules, remove it. + if err := ts.globalCell.Delete(ctx, KeyspaceRoutingRulesFile, nil); err != nil && !IsErrType(err, NoNode) { + return err + } + return nil + } + _, err = ts.globalCell.Update(ctx, KeyspaceRoutingRulesFile, data, nil) + return err +} + +func (ts *Server) GetKeyspaceRoutingRules(ctx context.Context) (*vschemapb.KeyspaceRoutingRules, error) { + rules := &vschemapb.KeyspaceRoutingRules{} + data, _, err := ts.globalCell.Get(ctx, KeyspaceRoutingRulesFile) + if err != nil { + if IsErrType(err, NoNode) { + return nil, nil + } + return nil, err + } + err = rules.UnmarshalVT(data) + if err != nil { + return nil, vterrors.Wrapf(err, "bad keyspace routing rules data: %q", data) + } + return rules, nil +} diff --git a/go/vt/topotools/routing_rules.go b/go/vt/topotools/routing_rules.go index 9eb64c936d7..663369344b0 100644 --- a/go/vt/topotools/routing_rules.go +++ b/go/vt/topotools/routing_rules.go @@ -27,7 +27,7 @@ import ( vschemapb "vitess.io/vitess/go/vt/proto/vschema" ) -//region routing rules +// region routing rules func GetRoutingRulesMap(rules *vschemapb.RoutingRules) map[string][]string { if rules == nil { @@ -69,9 +69,9 @@ func SaveRoutingRules(ctx context.Context, ts *topo.Server, rules map[string][]s return ts.SaveRoutingRules(ctx, rrs) } -//endregion +// endregion -//region shard routing rules +// region shard routing rules func GetShardRoutingRuleKey(fromKeyspace, shard string) string { return fmt.Sprintf("%s.%s", fromKeyspace, shard) @@ -122,3 +122,40 @@ func SaveShardRoutingRules(ctx context.Context, ts *topo.Server, srr map[string] return ts.SaveShardRoutingRules(ctx, srs) } + +// endregion + +// region keyspace routing rules + +func GetKeyspaceRoutingRulesMap(rules *vschemapb.KeyspaceRoutingRules) map[string]string { + if rules == nil { + return make(map[string]string) + } + rulesMap := make(map[string]string, len(rules.Rules)) + for _, rr := range rules.Rules { + rulesMap[rr.FromKeyspace] = rr.ToKeyspace + } + return rulesMap +} + +func GetKeyspaceRoutingRules(ctx context.Context, ts *topo.Server) (map[string]string, error) { + keyspaceRoutingRules, err := ts.GetKeyspaceRoutingRules(ctx) + if err != nil { + return nil, err + } + rules := GetKeyspaceRoutingRulesMap(keyspaceRoutingRules) + return rules, nil +} + +func SaveKeyspaceRoutingRules(ctx context.Context, ts *topo.Server, rules map[string]string) error { + keyspaceRoutingRules := &vschemapb.KeyspaceRoutingRules{Rules: make([]*vschemapb.KeyspaceRoutingRule, 0, len(rules))} + for from, to := range rules { + keyspaceRoutingRules.Rules = append(keyspaceRoutingRules.Rules, &vschemapb.KeyspaceRoutingRule{ + FromKeyspace: from, + ToKeyspace: to, + }) + } + return ts.SaveKeyspaceRoutingRules(ctx, keyspaceRoutingRules) +} + +// endregion diff --git a/go/vt/topotools/routing_rules_test.go b/go/vt/topotools/routing_rules_test.go index 0b4f265a77b..745c7b99025 100644 --- a/go/vt/topotools/routing_rules_test.go +++ b/go/vt/topotools/routing_rules_test.go @@ -90,3 +90,22 @@ func TestShardRoutingRulesRoundTrip(t *testing.T) { assert.Equal(t, srr, roundtripRules) } + +func TestKeyspaceRoutingRulesRoundTrip(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + ts := memorytopo.NewServer(ctx, "zone1") + defer ts.Close() + + rulesMap := map[string]string{ + "ks1": "ks2", + "ks4": "ks5", + } + + err := SaveKeyspaceRoutingRules(ctx, ts, rulesMap) + require.NoError(t, err, "could not save keyspace routing rules to topo %v", rulesMap) + + roundtripRulesMap, err := GetKeyspaceRoutingRules(ctx, ts) + require.NoError(t, err, "could not fetch keyspace routing rules from topo") + assert.EqualValues(t, rulesMap, roundtripRulesMap) +} diff --git a/go/vt/vtctl/grpcvtctldclient/client_gen.go b/go/vt/vtctl/grpcvtctldclient/client_gen.go index 78825e666b7..ff753111020 100644 --- a/go/vt/vtctl/grpcvtctldclient/client_gen.go +++ b/go/vt/vtctl/grpcvtctldclient/client_gen.go @@ -47,6 +47,15 @@ func (client *gRPCVtctldClient) AddCellsAlias(ctx context.Context, in *vtctldata return client.c.AddCellsAlias(ctx, in, opts...) } +// ApplyKeyspaceRoutingRules is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) ApplyKeyspaceRoutingRules(ctx context.Context, in *vtctldatapb.ApplyKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldatapb.ApplyKeyspaceRoutingRulesResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.ApplyKeyspaceRoutingRules(ctx, in, opts...) +} + // ApplyRoutingRules is part of the vtctlservicepb.VtctldClient interface. func (client *gRPCVtctldClient) ApplyRoutingRules(ctx context.Context, in *vtctldatapb.ApplyRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldatapb.ApplyRoutingRulesResponse, error) { if client.c == nil { @@ -326,6 +335,15 @@ func (client *gRPCVtctldClient) GetKeyspace(ctx context.Context, in *vtctldatapb return client.c.GetKeyspace(ctx, in, opts...) } +// GetKeyspaceRoutingRules is part of the vtctlservicepb.VtctldClient interface. +func (client *gRPCVtctldClient) GetKeyspaceRoutingRules(ctx context.Context, in *vtctldatapb.GetKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldatapb.GetKeyspaceRoutingRulesResponse, error) { + if client.c == nil { + return nil, status.Error(codes.Unavailable, connClosedMsg) + } + + return client.c.GetKeyspaceRoutingRules(ctx, in, opts...) +} + // GetKeyspaces is part of the vtctlservicepb.VtctldClient interface. func (client *gRPCVtctldClient) GetKeyspaces(ctx context.Context, in *vtctldatapb.GetKeyspacesRequest, opts ...grpc.CallOption) (*vtctldatapb.GetKeyspacesResponse, error) { if client.c == nil { diff --git a/go/vt/vtctl/grpcvtctldserver/server.go b/go/vt/vtctl/grpcvtctldserver/server.go index 079813ff666..8e69f8ce032 100644 --- a/go/vt/vtctl/grpcvtctldserver/server.go +++ b/go/vt/vtctl/grpcvtctldserver/server.go @@ -5165,3 +5165,43 @@ func (s *VtctldServer) diffVersion(ctx context.Context, primaryVersion string, p er.RecordError(fmt.Errorf("primary %v version %v is different than replica %v version %v", topoproto.TabletAliasString(primaryAlias), primaryVersion, topoproto.TabletAliasString(alias), replicaVersion)) } } + +// ApplyKeyspaceRoutingRules is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) ApplyKeyspaceRoutingRules(ctx context.Context, req *vtctldatapb.ApplyKeyspaceRoutingRulesRequest) (*vtctldatapb.ApplyKeyspaceRoutingRulesResponse, error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.ApplyKeyspaceRoutingRules") + defer span.Finish() + + span.Annotate("skip_rebuild", req.SkipRebuild) + span.Annotate("rebuild_cells", strings.Join(req.RebuildCells, ",")) + + if err := s.ts.SaveKeyspaceRoutingRules(ctx, req.KeyspaceRoutingRules); err != nil { + return nil, err + } + + resp := &vtctldatapb.ApplyKeyspaceRoutingRulesResponse{} + + if req.SkipRebuild { + return resp, nil + } + + if err := s.ts.RebuildSrvVSchema(ctx, req.RebuildCells); err != nil { + return nil, vterrors.Wrapf(err, "RebuildSrvVSchema(%v) failed: %v", req.RebuildCells, err) + } + + return resp, nil +} + +// GetKeyspaceRoutingRules is part of the vtctlservicepb.VtctldServer interface. +func (s *VtctldServer) GetKeyspaceRoutingRules(ctx context.Context, req *vtctldatapb.GetKeyspaceRoutingRulesRequest) (*vtctldatapb.GetKeyspaceRoutingRulesResponse, error) { + span, ctx := trace.NewSpan(ctx, "VtctldServer.GetKeyspaceRoutingRules") + defer span.Finish() + + rules, err := s.ts.GetKeyspaceRoutingRules(ctx) + if err != nil { + return nil, err + } + + return &vtctldatapb.GetKeyspaceRoutingRulesResponse{ + KeyspaceRoutingRules: rules, + }, nil +} diff --git a/go/vt/vtctl/localvtctldclient/client_gen.go b/go/vt/vtctl/localvtctldclient/client_gen.go index ecd4b8b7efd..e854514dcfa 100644 --- a/go/vt/vtctl/localvtctldclient/client_gen.go +++ b/go/vt/vtctl/localvtctldclient/client_gen.go @@ -39,6 +39,11 @@ func (client *localVtctldClient) AddCellsAlias(ctx context.Context, in *vtctldat return client.s.AddCellsAlias(ctx, in) } +// ApplyKeyspaceRoutingRules is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) ApplyKeyspaceRoutingRules(ctx context.Context, in *vtctldatapb.ApplyKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldatapb.ApplyKeyspaceRoutingRulesResponse, error) { + return client.s.ApplyKeyspaceRoutingRules(ctx, in) +} + // ApplyRoutingRules is part of the vtctlservicepb.VtctldClient interface. func (client *localVtctldClient) ApplyRoutingRules(ctx context.Context, in *vtctldatapb.ApplyRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldatapb.ApplyRoutingRulesResponse, error) { return client.s.ApplyRoutingRules(ctx, in) @@ -286,6 +291,11 @@ func (client *localVtctldClient) GetKeyspace(ctx context.Context, in *vtctldatap return client.s.GetKeyspace(ctx, in) } +// GetKeyspaceRoutingRules is part of the vtctlservicepb.VtctldClient interface. +func (client *localVtctldClient) GetKeyspaceRoutingRules(ctx context.Context, in *vtctldatapb.GetKeyspaceRoutingRulesRequest, opts ...grpc.CallOption) (*vtctldatapb.GetKeyspaceRoutingRulesResponse, error) { + return client.s.GetKeyspaceRoutingRules(ctx, in) +} + // GetKeyspaces is part of the vtctlservicepb.VtctldClient interface. func (client *localVtctldClient) GetKeyspaces(ctx context.Context, in *vtctldatapb.GetKeyspacesRequest, opts ...grpc.CallOption) (*vtctldatapb.GetKeyspacesResponse, error) { return client.s.GetKeyspaces(ctx, in) diff --git a/go/vt/vtctl/workflow/materializer.go b/go/vt/vtctl/workflow/materializer.go index bdab5f84d09..93eb7bd9327 100644 --- a/go/vt/vtctl/workflow/materializer.go +++ b/go/vt/vtctl/workflow/materializer.go @@ -18,6 +18,7 @@ package workflow import ( "context" + "encoding/json" "fmt" "strings" "sync" @@ -96,6 +97,24 @@ func (mz *materializer) getWorkflowSubType() (binlogdatapb.VReplicationWorkflowS } } +func (mz *materializer) getOptionsJSON() (string, error) { + vrOptions := &vtctldatapb.WorkflowOptions{} + if mz.IsMultiTenantMigration() { + vrOptions.TenantId = mz.ms.WorkflowOptions.TenantId + if mz.ms.WorkflowOptions.SourceKeyspaceAlias != "" { + vrOptions.SourceKeyspaceAlias = mz.ms.WorkflowOptions.SourceKeyspaceAlias + } + } + optionsJSON, err := json.Marshal(vrOptions) + if err != nil { + return "", err + } + if optionsJSON == nil { + optionsJSON = []byte("{}") + } + return string(optionsJSON), nil +} + func (mz *materializer) createWorkflowStreams(req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) error { if err := validateNewWorkflow(mz.ctx, mz.ts, mz.tmc, mz.ms.TargetKeyspace, mz.ms.Workflow); err != nil { return err @@ -114,6 +133,11 @@ func (mz *materializer) createWorkflowStreams(req *tabletmanagerdatapb.CreateVRe return err } req.WorkflowSubType = workflowSubType + optionsJSON, err := mz.getOptionsJSON() + if err != nil { + return err + } + req.Options = optionsJSON return mz.forAllTargets(func(target *topo.ShardInfo) error { targetPrimary, err := mz.ts.GetTablet(mz.ctx, target.PrimaryAlias) @@ -132,6 +156,7 @@ func (mz *materializer) createWorkflowStreams(req *tabletmanagerdatapb.CreateVRe if len(sourceShards) == 1 && key.KeyRangeEqual(sourceShards[0].KeyRange, target.KeyRange) { streamKeyRangesEqual = true } + // Each tablet needs its own copy of the request as it will have a unique // BinlogSource. tabletReq := req.CloneVT() @@ -145,6 +170,10 @@ func (mz *materializer) createWorkflowStreams(req *tabletmanagerdatapb.CreateVRe }) } +func (mz *materializer) getTenantClause() (*sqlparser.Expr, error) { + return getTenantClause(mz.ms.WorkflowOptions, mz.targetVSchema, mz.env.Parser()) +} + func (mz *materializer) generateBinlogSources(ctx context.Context, targetShard *topo.ShardInfo, sourceShards []*topo.ShardInfo, keyRangesEqual bool) ([]*binlogdatapb.BinlogSource, error) { blses := make([]*binlogdatapb.BinlogSource, 0, len(mz.sourceShards)) for _, sourceShard := range sourceShards { @@ -158,6 +187,16 @@ func (mz *materializer) generateBinlogSources(ctx context.Context, targetShard * TargetTimeZone: mz.ms.TargetTimeZone, OnDdl: binlogdatapb.OnDDLAction(binlogdatapb.OnDDLAction_value[mz.ms.OnDdl]), } + + var tenantClause *sqlparser.Expr + var err error + if mz.IsMultiTenantMigration() { + tenantClause, err = mz.getTenantClause() + if err != nil { + return nil, err + } + } + for _, ts := range mz.ms.TableSettings { rule := &binlogdatapb.Rule{ Match: ts.TargetTable, @@ -177,7 +216,6 @@ func (mz *materializer) generateBinlogSources(ctx context.Context, targetShard * if !ok { return nil, fmt.Errorf("unrecognized statement: %s", ts.SourceExpression) } - filter := ts.SourceExpression if !keyRangesEqual && mz.targetVSchema.Keyspace.Sharded && mz.targetVSchema.Tables[ts.TargetTable].Type != vindexes.TypeReference { cv, err := vindexes.FindBestColVindex(mz.targetVSchema.Tables[ts.TargetTable]) if err != nil { @@ -202,25 +240,12 @@ func (mz *materializer) generateBinlogSources(ctx context.Context, targetShard * Name: sqlparser.NewIdentifierCI("in_keyrange"), Exprs: subExprs, } - if sel.Where != nil { - sel.Where = &sqlparser.Where{ - Type: sqlparser.WhereClause, - Expr: &sqlparser.AndExpr{ - Left: inKeyRange, - Right: sel.Where.Expr, - }, - } - } else { - sel.Where = &sqlparser.Where{ - Type: sqlparser.WhereClause, - Expr: inKeyRange, - } - } - - filter = sqlparser.String(sel) + addFilter(sel, inKeyRange) } - - rule.Filter = filter + if tenantClause != nil { + addFilter(sel, *tenantClause) + } + rule.Filter = sqlparser.String(sel) bls.Filter.Rules = append(bls.Filter.Rules, rule) } blses = append(blses, bls) @@ -616,3 +641,10 @@ func primaryVindexesDiffer(ms *vtctldatapb.MaterializeSettings, source, target * } return false } + +func (mz *materializer) IsMultiTenantMigration() bool { + if mz.ms.WorkflowOptions != nil && mz.ms.WorkflowOptions.TenantId != "" { + return true + } + return false +} diff --git a/go/vt/vtctl/workflow/materializer_env_test.go b/go/vt/vtctl/workflow/materializer_env_test.go index e841be49f0e..587712f620c 100644 --- a/go/vt/vtctl/workflow/materializer_env_test.go +++ b/go/vt/vtctl/workflow/materializer_env_test.go @@ -52,13 +52,11 @@ type queryResult struct { } type testMaterializerEnv struct { - ws *Server - ms *vtctldatapb.MaterializeSettings - sources []string - targets []string - tablets map[int]*topodatapb.Tablet - // Importing the tabletmanager package causes a circular dependency. :-( - //tms map[int]*tabletmanager.TabletManager + ws *Server + ms *vtctldatapb.MaterializeSettings + sources []string + targets []string + tablets map[int]*topodatapb.Tablet topoServ *topo.Server cell string tmc *testMaterializerTMClient diff --git a/go/vt/vtctl/workflow/materializer_test.go b/go/vt/vtctl/workflow/materializer_test.go index 4f26a2bf626..0041c2fa515 100644 --- a/go/vt/vtctl/workflow/materializer_test.go +++ b/go/vt/vtctl/workflow/materializer_test.go @@ -45,13 +45,11 @@ import ( const ( position = "9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97" - mzUpdateQuery = "update _vt.vreplication set state='Running' where db_name='vt_targetks' and workflow='workflow'" mzSelectFrozenQuery = "select 1 from _vt.vreplication where db_name='vt_targetks' and message='FROZEN' and workflow_sub_type != 1" mzCheckJournal = "/select val from _vt.resharding_journal where id=" mzGetCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" mzGetLatestCopyState = "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1) and id in (select max(id) from _vt.copy_state where vrepl_id in (1) group by vrepl_id, table_name)" - insertPrefix = `/insert into _vt.vreplication\(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys\) values ` - eol = "$" + insertPrefix = `/insert into _vt.vreplication\(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options\) values ` ) var ( @@ -475,8 +473,8 @@ func TestMoveTablesDDLFlag(t *testing.T) { defer env.close() // This is the default and go does not marshal defaults // for prototext fields so we use the default insert stmt. - //insert = fmt.Sprintf(`/insert into .vreplication\(.*on_ddl:%s.*`, onDDLAction) - //env.tmc.expectVRQuery(100, "/.*", &sqltypes.Result{}) + // insert = fmt.Sprintf(`/insert into .vreplication\(.*on_ddl:%s.*`, onDDLAction) + // env.tmc.expectVRQuery(100, "/.*", &sqltypes.Result{}) // TODO: we cannot test the actual query generated w/o having a // TabletManager. Importing the tabletmanager package, however, causes @@ -524,8 +522,8 @@ func TestMoveTablesNoRoutingRules(t *testing.T) { defer env.close() // This is the default and go does not marshal defaults // for prototext fields so we use the default insert stmt. - //insert = fmt.Sprintf(`/insert into .vreplication\(.*on_ddl:%s.*`, onDDLAction) - //env.tmc.expectVRQuery(100, "/.*", &sqltypes.Result{}) + // insert = fmt.Sprintf(`/insert into .vreplication\(.*on_ddl:%s.*`, onDDLAction) + // env.tmc.expectVRQuery(100, "/.*", &sqltypes.Result{}) // TODO: we cannot test the actual query generated w/o having a // TabletManager. Importing the tabletmanager package, however, causes @@ -2246,6 +2244,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { }, }, }, + Options: "{}", }, }, }, @@ -2279,6 +2278,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { }, }, }, + Options: "{}", }, 210: { Workflow: workflow, @@ -2298,6 +2298,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { }, }, }, + Options: "{}", }, }, }, @@ -2343,6 +2344,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { }, }, }, + Options: "{}", }, }, }, @@ -2376,6 +2378,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { }, }, }, + Options: "{}", }, 210: { Workflow: workflow, @@ -2395,6 +2398,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { }, }, }, + Options: "{}", }, }, }, @@ -2427,7 +2431,6 @@ func TestKeyRangesEqualOptimization(t *testing.T) { if tablet.Keyspace != targetKs || tablet.Type != topodatapb.TabletType_PRIMARY { continue } - env.tmc.expectVRQuery(int(tablet.Alias.Uid), mzSelectFrozenQuery, &sqltypes.Result{}) // If we are doing a partial MoveTables, we will only perform the workflow // stream creation / INSERT statment on the shard(s) we're migrating. if len(tc.moveTablesReq.SourceShards) > 0 && !slices.Contains(tc.moveTablesReq.SourceShards, tablet.Shard) { @@ -2454,6 +2457,7 @@ func TestKeyRangesEqualOptimization(t *testing.T) { DeferSecondaryKeys: tc.moveTablesReq.DeferSecondaryKeys, AutoStart: tc.moveTablesReq.AutoStart, StopAfterCopy: tc.moveTablesReq.StopAfterCopy, + Options: "{}", }) require.NoError(t, err, "createWorkflowStreams failed: %v", err) }) diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index c29cd3228db..1928ecb5e5f 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -18,6 +18,7 @@ package workflow import ( "context" + "encoding/json" "errors" "fmt" "math" @@ -524,7 +525,12 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows for i := range cells { cells[i] = strings.TrimSpace(cells[i]) } - + options := res.Options + if options != "" { + if err := json.Unmarshal([]byte(options), &workflow.Options); err != nil { + return err + } + } stream := &vtctldatapb.Workflow_Stream{ Id: int64(rstream.Id), Shard: tablet.Shard, @@ -971,7 +977,19 @@ func (s *Server) getWorkflowState(ctx context.Context, targetKeyspace, workflowN } table := ts.Tables()[0] - if ts.isPartialMigration { // shard level traffic switching is all or nothing + if ts.IsMultiTenantMigration() { // traffic switching for multi-tenant migrations is all or nothing + keyspaceRoutingRules, err := topotools.GetKeyspaceRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return nil, nil, err + } + currentTargetKeyspace := keyspaceRoutingRules[ts.sourceKeyspace] + if currentTargetKeyspace == ts.targetKeyspace { + log.Infof("Keyspace routing rules: routing currently to target, so marking all traffic as switched") + state.WritesSwitched = true + state.ReplicaCellsNotSwitched = nil + state.RdonlyCellsNotSwitched = nil + } + } else if ts.isPartialMigration { // shard level traffic switching is all or nothing shardRoutingRules, err := s.ts.GetShardRoutingRules(ctx) if err != nil { return nil, nil, err @@ -1317,6 +1335,19 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl if vschema == nil { return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no vschema found for target keyspace %s", targetKeyspace) } + + if workflowType == binlogdatapb.VReplicationWorkflowType_MoveTables && + req.GetWorkflowOptions().GetTenantId() != "" { + multiTenantSpec := vschema.MultiTenantSpec + if multiTenantSpec == nil { + return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "multi-tenant spec not found for target keyspace %s", targetKeyspace) + } + // Validate that the tenant id matches the data type of the column provided in the multi-tenant spec of the vschema. + if err := validateTenantId(multiTenantSpec.TenantIdColumnType, req.WorkflowOptions.TenantId); err != nil { + return nil, err + } + } + ksTables, err := getTablesInKeyspace(ctx, sourceTopo, s.tmc, sourceKeyspace) if err != nil { return nil, err @@ -1373,6 +1404,7 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl OnDdl: req.OnDdl, DeferSecondaryKeys: req.DeferSecondaryKeys, AtomicCopy: req.AtomicCopy, + WorkflowOptions: req.WorkflowOptions, } if req.SourceTimeZone != "" { ms.SourceTimeZone = req.SourceTimeZone @@ -1438,36 +1470,8 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl // Now that the streams have been successfully created, let's put the associated // routing rules in place. if externalTopo == nil { - if req.NoRoutingRules { - log.Warningf("Found --no-routing-rules flag, not creating routing rules for workflow %s.%s", targetKeyspace, req.Workflow) - } else { - // Save routing rules before vschema. If we save vschema first, and routing - // rules fails to save, we may generate duplicate table errors. - if mz.isPartial { - if err := createDefaultShardRoutingRules(mz.ctx, mz.ms, mz.ts); err != nil { - return nil, err - } - } - - rules, err := topotools.GetRoutingRules(ctx, s.ts) - if err != nil { - return nil, err - } - for _, table := range tables { - toSource := []string{sourceKeyspace + "." + table} - rules[table] = toSource - rules[table+"@replica"] = toSource - rules[table+"@rdonly"] = toSource - rules[targetKeyspace+"."+table] = toSource - rules[targetKeyspace+"."+table+"@replica"] = toSource - rules[targetKeyspace+"."+table+"@rdonly"] = toSource - rules[targetKeyspace+"."+table] = toSource - rules[sourceKeyspace+"."+table+"@replica"] = toSource - rules[sourceKeyspace+"."+table+"@rdonly"] = toSource - } - if err := topotools.SaveRoutingRules(ctx, s.ts, rules); err != nil { - return nil, err - } + if err := s.setupInitialRoutingRules(ctx, req, mz, tables, vschema); err != nil { + return nil, err } // We added to the vschema. @@ -1522,6 +1526,88 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl }) } +func (s *Server) validateRoutingRuleFlags(req *vtctldatapb.MoveTablesCreateRequest, mz *materializer) error { + if mz.IsMultiTenantMigration() { + switch { + case req.NoRoutingRules: + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cannot use --no-routing-rules in a multi-tenant migration") + case mz.isPartial: + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cannot run partial shard migration along with multi-tenant migration") + } + } + return nil +} + +func (s *Server) setupInitialRoutingRules(ctx context.Context, req *vtctldatapb.MoveTablesCreateRequest, mz *materializer, tables []string, vschema *vschemapb.Keyspace) error { + if err := s.validateRoutingRuleFlags(req, mz); err != nil { + return err + } + + const ( + primaryType = "" + globalRoute + ) + sourceKeyspace := req.SourceKeyspace + targetKeyspace := req.TargetKeyspace + + if req.NoRoutingRules { + log.Warningf("Found --no-routing-rules flag, not creating routing rules for workflow %s.%s", targetKeyspace, req.Workflow) + return nil + } + + // Save routing rules before vschema. If we save vschema first, and routing + // rules fails to save, we may generate duplicate table errors. + if mz.isPartial { + if err := createDefaultShardRoutingRules(mz.ctx, mz.ms, mz.ts); err != nil { + return err + } + } + + if mz.IsMultiTenantMigration() { + log.Infof("Setting up keyspace routing rules for workflow %s.%s", targetKeyspace, req.Workflow) + var keyspaces []string + // Note that you can never point the target keyspace to the source keyspace in a multi-tenant migration + // since the target takes write traffic for all tenants! + keyspaces = append(keyspaces, sourceKeyspace) + if req.GetWorkflowOptions().GetSourceKeyspaceAlias() != "" { + keyspaces = append(keyspaces, req.WorkflowOptions.SourceKeyspaceAlias) + } + routes := make(map[string]string) + for _, ks := range keyspaces { + routes[ks] = sourceKeyspace + } + if err := updateKeyspaceRoutingRule(ctx, s.ts, routes); err != nil { + return err + } + return nil + } + + // Setup table routing rules. + rules, err := topotools.GetRoutingRules(ctx, s.ts) + if err != nil { + return err + } + routeTableToSource := func(keyspace, table string) { + key := table + route := fmt.Sprintf("%s.%s", sourceKeyspace, table) + if keyspace != "" { + key = fmt.Sprintf("%s.%s", keyspace, table) + } + for _, typ := range []string{primaryType, "@replica", "@rdonly"} { + rules[key+typ] = []string{route} + } + } + for _, table := range tables { + for _, ks := range []string{globalRoute, targetKeyspace, sourceKeyspace} { + routeTableToSource(ks, table) + } + } + if err := topotools.SaveRoutingRules(ctx, s.ts, rules); err != nil { + return err + } + return nil +} + // MoveTablesComplete is part of the vtctlservicepb.VtctldServer interface. // It cleans up a successful MoveTables workflow and its related artifacts. // Note: this is currently re-used for Reshard as well. @@ -2498,6 +2584,7 @@ func (s *Server) buildTrafficSwitcher(ctx context.Context, targetKeyspace, workf optTabletTypes: optTabletTypes, workflowType: tgtInfo.WorkflowType, workflowSubType: tgtInfo.WorkflowSubType, + options: tgtInfo.Options, } log.Infof("Migration ID for workflow %s: %d", workflowName, ts.id) sourceTopo := s.ts @@ -2687,6 +2774,10 @@ func (s *Server) dropArtifacts(ctx context.Context, keepRoutingRules bool, sw is if err := sw.deleteShardRoutingRules(ctx); err != nil { return err } + if err := sw.deleteKeyspaceRoutingRules(ctx); err != nil { + return err + } + } return nil @@ -2917,7 +3008,20 @@ func (s *Server) WorkflowSwitchTraffic(ctx context.Context, req *vtctldatapb.Wor if err != nil { return nil, err } + if ts.IsMultiTenantMigration() { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "cannot reverse traffic for multi-tenant migrations") + } + } + hasReplica, hasRdonly, hasPrimary, err = parseTabletTypes(req.TabletTypes) + if err != nil { + return nil, err + } + if ts.IsMultiTenantMigration() && !(hasRdonly && hasReplica && hasPrimary) { + err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, + "for multi-tenant migrations, all traffic needs to be switched at once for workflow %s", req.Workflow) + return nil, err } + reason, err := s.canSwitch(ctx, ts, startState, direction, int64(maxReplicationLagAllowed.Seconds()), req.Shards) if err != nil { return nil, err @@ -2979,6 +3083,7 @@ func (s *Server) WorkflowSwitchTraffic(ctx context.Context, req *vtctldatapb.Wor } else { resp.CurrentState = currentState.String() } + log.Infof("SwitchTraffic done for workflow %s.%s, returning response %v", req.Keyspace, req.Workflow, resp) } return resp, nil } @@ -3018,7 +3123,20 @@ func (s *Server) switchReads(ctx context.Context, req *vtctldatapb.WorkflowSwitc if !switchReplica && !switchRdonly { return handleError("invalid tablet types", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "tablet types must be REPLICA or RDONLY: %s", roTypesToSwitchStr)) } - if !ts.isPartialMigration { // shard level traffic switching is all or nothing + // For partial (shard-by-shard migrations) or multi-tenant migrations, traffic for all tablet types + // is expected to be switched at once. For other MoveTables migrations where we use table routing rules + // replica/rdonly traffic can be switched first and then primary traffic can be switched later. + trafficSwitchingIsAllOrNothing := false + switch { + case ts.isPartialMigration: + // shard level traffic switching is all or nothing + trafficSwitchingIsAllOrNothing = true + case ts.MigrationType() == binlogdatapb.MigrationType_TABLES && ts.IsMultiTenantMigration(): + // keyspace routing rules are used, traffic is all or nothing per keyspace + trafficSwitchingIsAllOrNothing = true + } + + if !trafficSwitchingIsAllOrNothing { if direction == DirectionBackward && switchReplica && len(state.ReplicaCellsSwitched) == 0 { return handleError("invalid request", vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "requesting reversal of read traffic for REPLICAs but REPLICA reads have not been switched")) } @@ -3069,10 +3187,16 @@ func (s *Server) switchReads(ctx context.Context, req *vtctldatapb.WorkflowSwitc defer unlock(&err) if ts.MigrationType() == binlogdatapb.MigrationType_TABLES { - if ts.isPartialMigration { + switch { + case ts.IsMultiTenantMigration(): + ts.Logger().Infof("If keyspace routing rules are used, traffic is all or nothing per keyspace for workflow %s.%s", ts.targetKeyspace, ts.workflow) + case ts.isPartialMigration: ts.Logger().Infof("Partial migration, skipping switchTableReads as traffic is all or nothing per shard and overridden for reads AND writes in the ShardRoutingRule created when switching writes.") - } else if err := sw.switchTableReads(ctx, req.Cells, roTabletTypes, direction); err != nil { - return handleError("failed to switch read traffic for the tables", err) + default: + err := sw.switchTableReads(ctx, req.Cells, roTabletTypes, direction) + if err != nil { + return handleError("failed to switch read traffic for the tables", err) + } } return sw.logs(), nil } diff --git a/go/vt/vtctl/workflow/switcher.go b/go/vt/vtctl/workflow/switcher.go index 0cbdce164dc..59bb387cca5 100644 --- a/go/vt/vtctl/workflow/switcher.go +++ b/go/vt/vtctl/workflow/switcher.go @@ -42,6 +42,10 @@ func (r *switcher) deleteShardRoutingRules(ctx context.Context) error { return r.ts.deleteShardRoutingRules(ctx) } +func (r *switcher) deleteKeyspaceRoutingRules(ctx context.Context) error { + return r.ts.deleteKeyspaceRoutingRules(ctx) +} + func (r *switcher) dropSourceDeniedTables(ctx context.Context) error { return r.ts.dropSourceDeniedTables(ctx) } diff --git a/go/vt/vtctl/workflow/switcher_dry_run.go b/go/vt/vtctl/workflow/switcher_dry_run.go index 21b975a0d6b..eb6ab80d351 100644 --- a/go/vt/vtctl/workflow/switcher_dry_run.go +++ b/go/vt/vtctl/workflow/switcher_dry_run.go @@ -56,6 +56,13 @@ func (dr *switcherDryRun) deleteShardRoutingRules(ctx context.Context) error { return nil } +func (dr *switcherDryRun) deleteKeyspaceRoutingRules(ctx context.Context) error { + if dr.ts.IsMultiTenantMigration() { + dr.drLog.Log("Keyspace routing rules will be deleted") + } + return nil +} + func (dr *switcherDryRun) switchShardReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error { sourceShards := make([]string, 0) targetShards := make([]string, 0) diff --git a/go/vt/vtctl/workflow/switcher_interface.go b/go/vt/vtctl/workflow/switcher_interface.go index 8d0f9e847be..82e3a2be357 100644 --- a/go/vt/vtctl/workflow/switcher_interface.go +++ b/go/vt/vtctl/workflow/switcher_interface.go @@ -50,6 +50,7 @@ type iswitcher interface { dropTargetShards(ctx context.Context) error deleteRoutingRules(ctx context.Context) error deleteShardRoutingRules(ctx context.Context) error + deleteKeyspaceRoutingRules(ctx context.Context) error addParticipatingTablesToKeyspace(ctx context.Context, keyspace, tableSpecs string) error resetSequences(ctx context.Context) error initializeTargetSequences(ctx context.Context, sequencesByBackingTable map[string]*sequenceMetadata) error diff --git a/go/vt/vtctl/workflow/traffic_switcher.go b/go/vt/vtctl/workflow/traffic_switcher.go index 2f6e0659025..3cdffda4993 100644 --- a/go/vt/vtctl/workflow/traffic_switcher.go +++ b/go/vt/vtctl/workflow/traffic_switcher.go @@ -18,6 +18,7 @@ package workflow import ( "context" + "encoding/json" "errors" "fmt" "sort" @@ -25,6 +26,8 @@ import ( "sync" "time" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" + "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" @@ -174,6 +177,7 @@ type TargetInfo struct { OptTabletTypes string WorkflowType binlogdatapb.VReplicationWorkflowType WorkflowSubType binlogdatapb.VReplicationWorkflowSubType + Options *vtctldatapb.WorkflowOptions } // MigrationSource contains the metadata for each migration source. @@ -234,6 +238,7 @@ type trafficSwitcher struct { targetTimeZone string workflowType binlogdatapb.VReplicationWorkflowType workflowSubType binlogdatapb.VReplicationWorkflowSubType + options *vtctldatapb.WorkflowOptions } func (ts *trafficSwitcher) TopoServer() *topo.Server { return ts.ws.ts } @@ -434,6 +439,27 @@ func (ts *trafficSwitcher) deleteShardRoutingRules(ctx context.Context) error { return nil } +func (ts *trafficSwitcher) deleteKeyspaceRoutingRules(ctx context.Context) error { + log.Infof("deleteKeyspaceRoutingRules: workflow %s.%s", ts.targetKeyspace, ts.workflow) + if !ts.IsMultiTenantMigration() { + return nil + } + krr, err := topotools.GetKeyspaceRoutingRules(ctx, ts.TopoServer()) + if err != nil { + return err + } + log.Infof("deleteKeyspaceRoutingRules before: %s", krr) + delete(krr, ts.SourceKeyspaceName()) + if ts.options.SourceKeyspaceAlias != "" { + delete(krr, ts.options.SourceKeyspaceAlias) + } + log.Infof("deleteKeyspaceRoutingRules after: %s", krr) + if err := topotools.SaveKeyspaceRoutingRules(ctx, ts.TopoServer(), krr); err != nil { + return err + } + return nil +} + func (ts *trafficSwitcher) dropSourceDeniedTables(ctx context.Context) error { return ts.ForAllSources(func(source *MigrationSource) error { if _, err := ts.TopoServer().UpdateShardFields(ctx, ts.SourceKeyspaceName(), source.GetShard().ShardName(), func(si *topo.ShardInfo) error { @@ -708,7 +734,21 @@ func (ts *trafficSwitcher) changeRouting(ctx context.Context) error { } func (ts *trafficSwitcher) changeWriteRoute(ctx context.Context) error { - if ts.isPartialMigration { + if ts.IsMultiTenantMigration() { + ts.Logger().Infof("Pointing keyspace routing rules to %s for workflow %s", ts.TargetKeyspaceName(), ts.workflow) + var keyspaces []string + keyspaces = append(keyspaces, ts.SourceKeyspaceName()) + if ts.options.SourceKeyspaceAlias != "" { + keyspaces = append(keyspaces, ts.options.SourceKeyspaceAlias) + } + routes := make(map[string]string) + for _, ks := range keyspaces { + routes[ks] = ts.TargetKeyspaceName() + } + if err := updateKeyspaceRoutingRule(ctx, ts.TopoServer(), routes); err != nil { + return err + } + } else if ts.isPartialMigration { srr, err := topotools.GetShardRoutingRules(ctx, ts.TopoServer()) if err != nil { return err @@ -782,7 +822,7 @@ func (ts *trafficSwitcher) changeShardRouting(ctx context.Context) error { return nil } -func (ts *trafficSwitcher) getReverseVReplicationUpdateQuery(targetCell string, sourceCell string, dbname string) string { +func (ts *trafficSwitcher) getReverseVReplicationUpdateQuery(targetCell string, sourceCell string, dbname string, options string) string { // we try to be clever to understand what user intends: // if target's cell is present in cells but not source's cell we replace it // with the source's cell. @@ -792,8 +832,8 @@ func (ts *trafficSwitcher) getReverseVReplicationUpdateQuery(targetCell string, } if ts.optCells != "" || ts.optTabletTypes != "" { - query := fmt.Sprintf("update _vt.vreplication set cell = '%s', tablet_types = '%s' where workflow = '%s' and db_name = '%s'", - ts.optCells, ts.optTabletTypes, ts.ReverseWorkflowName(), dbname) + query := fmt.Sprintf("update _vt.vreplication set cell = '%s', tablet_types = '%s', options = '%s' where workflow = '%s' and db_name = '%s'", + ts.optCells, ts.optTabletTypes, options, ts.ReverseWorkflowName(), dbname) return query } return "" @@ -846,7 +886,7 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error SourceTimeZone: bls.TargetTimeZone, TargetTimeZone: bls.SourceTimeZone, } - + var err error for _, rule := range bls.Filter.Rules { if rule.Filter == "exclude" { reverseBls.Filter.Rules = append(reverseBls.Filter.Rules, rule) @@ -883,6 +923,12 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error } } filter = fmt.Sprintf("select * from %s%s", sqlescape.EscapeID(rule.Match), inKeyrange) + if ts.IsMultiTenantMigration() { + filter, err = ts.addTenantFilter(ctx, filter) + if err != nil { + return err + } + } } reverseBls.Filter.Rules = append(reverseBls.Filter.Rules, &binlogdatapb.Rule{ Match: rule.Match, @@ -891,7 +937,7 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error } log.Infof("Creating reverse workflow vreplication stream on tablet %s: workflow %s, startPos %s", source.GetPrimary().Alias, ts.ReverseWorkflowName(), target.Position) - _, err := ts.VReplicationExec(ctx, source.GetPrimary().Alias, + _, err = ts.VReplicationExec(ctx, source.GetPrimary().Alias, binlogplayer.CreateVReplicationState(ts.ReverseWorkflowName(), reverseBls, target.Position, binlogdatapb.VReplicationWorkflowState_Stopped, source.GetPrimary().DbName(), ts.workflowType, ts.workflowSubType)) if err != nil { @@ -899,7 +945,12 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error } // if user has defined the cell/tablet_types parameters in the forward workflow, update the reverse workflow as well - updateQuery := ts.getReverseVReplicationUpdateQuery(target.GetPrimary().Alias.Cell, source.GetPrimary().Alias.Cell, source.GetPrimary().DbName()) + optionsJSON, err := json.Marshal(ts.options) + if err != nil { + return err + } + updateQuery := ts.getReverseVReplicationUpdateQuery(target.GetPrimary().Alias.Cell, + source.GetPrimary().Alias.Cell, source.GetPrimary().DbName(), string(optionsJSON)) if updateQuery != "" { log.Infof("Updating vreplication stream entry on %s with: %s", source.GetPrimary().Alias, updateQuery) _, err = ts.VReplicationExec(ctx, source.GetPrimary().Alias, updateQuery) @@ -910,6 +961,33 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error return err } +func (ts *trafficSwitcher) addTenantFilter(ctx context.Context, filter string) (string, error) { + parser := ts.ws.env.Parser() + vschema, err := ts.TopoServer().GetVSchema(ctx, ts.targetKeyspace) + if err != nil { + return "", err + } + targetVSchema, err := vindexes.BuildKeyspaceSchema(vschema, ts.targetKeyspace, parser) + if err != nil { + return "", err + } + tenantClause, err := getTenantClause(ts.options, targetVSchema, parser) + if err != nil { + return "", err + } + stmt, err := parser.Parse(filter) + if err != nil { + return "", err + } + sel, ok := stmt.(*sqlparser.Select) + if !ok { + return "", fmt.Errorf("unrecognized statement: %s", filter) + } + addFilter(sel, *tenantClause) + filter = sqlparser.String(sel) + return filter, nil +} + func (ts *trafficSwitcher) waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error { ctx, cancel := context.WithTimeout(ctx, filteredReplicationWaitTime) defer cancel() @@ -1561,3 +1639,10 @@ func (ts *trafficSwitcher) resetSequences(ctx context.Context) error { return ts.TabletManagerClient().ResetSequences(ctx, source.GetPrimary().Tablet, ts.Tables()) }) } + +func (ts *trafficSwitcher) IsMultiTenantMigration() bool { + if ts.options != nil && ts.options.TenantId != "" { + return true + } + return false +} diff --git a/go/vt/vtctl/workflow/utils.go b/go/vt/vtctl/workflow/utils.go index aef80d91c18..609a3d59f9f 100644 --- a/go/vt/vtctl/workflow/utils.go +++ b/go/vt/vtctl/workflow/utils.go @@ -19,13 +19,19 @@ package workflow import ( "bytes" "context" + "encoding/json" "fmt" "hash/fnv" "math" "sort" + "strconv" "strings" "sync" + querypb "vitess.io/vitess/go/vt/proto/query" + + "vitess.io/vitess/go/vt/vtgate/vindexes" + "google.golang.org/protobuf/encoding/prototext" "vitess.io/vitess/go/sets" @@ -120,12 +126,6 @@ func validateNewWorkflow(ctx context.Context, ts *topo.Server, tmc tmclient.Tabl allErrors.RecordError(fmt.Errorf("workflow %s already exists in keyspace %s on tablet %v", workflow, keyspace, primary.Alias)) return } - for _, stream := range wf.Streams { - if stream.Message == Frozen && wf.WorkflowSubType != binlogdatapb.VReplicationWorkflowSubType_Partial { - allErrors.RecordError(fmt.Errorf("found previous frozen workflow on tablet %v, please review and delete it first before creating a new workflow", primary.Alias)) - return - } - } } }(si) } @@ -337,6 +337,7 @@ func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManag targets = make(map[string]*MigrationTarget, len(targetShards)) workflowType binlogdatapb.VReplicationWorkflowType workflowSubType binlogdatapb.VReplicationWorkflowSubType + options vtctldatapb.WorkflowOptions ) // We check all shards in the target keyspace. Not all of them may have a @@ -375,6 +376,13 @@ func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManag optTabletTypes = topoproto.MakeStringTypeCSV(wf.TabletTypes) workflowType = wf.WorkflowType workflowSubType = wf.WorkflowSubType + optionsJSON := wf.GetOptions() + if optionsJSON != "" { + if err := json.Unmarshal([]byte(optionsJSON), &options); err != nil { + log.Errorf("failed to unmarshal options: %v %s", err, optionsJSON) + return nil, err + } + } for _, stream := range wf.Streams { if stream.Message == Frozen { @@ -397,6 +405,7 @@ func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManag OptTabletTypes: optTabletTypes, WorkflowType: workflowType, WorkflowSubType: workflowSubType, + Options: &options, }, nil } @@ -765,3 +774,92 @@ func LegacyBuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.Table WorkflowSubType: workflowSubType, }, nil } + +func addFilter(sel *sqlparser.Select, filter sqlparser.Expr) { + if sel.Where != nil { + sel.Where = &sqlparser.Where{ + Type: sqlparser.WhereClause, + Expr: &sqlparser.AndExpr{ + Left: filter, + Right: sel.Where.Expr, + }, + } + } else { + sel.Where = &sqlparser.Where{ + Type: sqlparser.WhereClause, + Expr: filter, + } + } +} + +func getTenantClause(vrOptions *vtctldatapb.WorkflowOptions, + targetVSchema *vindexes.KeyspaceSchema, parser *sqlparser.Parser) (*sqlparser.Expr, error) { + if vrOptions.TenantId == "" { + return nil, nil + } + if targetVSchema == nil || targetVSchema.MultiTenantSpec == nil { + return nil, fmt.Errorf("target keyspace not defined, or it does not have multi-tenant spec") + } + tenantColumnName := targetVSchema.MultiTenantSpec.TenantIdColumnName + tenantColumnType := targetVSchema.MultiTenantSpec.TenantIdColumnType + if tenantColumnName == "" { + return nil, fmt.Errorf("tenant column name not defined in multi-tenant spec") + } + + var tenantId string + switch tenantColumnType { + case querypb.Type_INT64: + _, err := strconv.Atoi(vrOptions.TenantId) + if err != nil { + return nil, fmt.Errorf("tenant id is not a valid int: %s", vrOptions.TenantId) + } + tenantId = vrOptions.TenantId + case querypb.Type_VARCHAR: + tenantId = fmt.Sprintf("'%s'", vrOptions.TenantId) + default: + return nil, fmt.Errorf("unsupported tenant column type: %s", tenantColumnType) + } + + stmt, err := parser.Parse(fmt.Sprintf("select * from t where %s = %s", tenantColumnName, tenantId)) + if err != nil { + return nil, err + } + sel, ok := stmt.(*sqlparser.Select) + if !ok { + return nil, fmt.Errorf("error getting select: %s", tenantId) + } + return &sel.Where.Expr, nil +} + +// updateKeyspaceRoutingRule updates the keyspace routing rule for the (effective) source keyspace to the target keyspace. +func updateKeyspaceRoutingRule(ctx context.Context, ts *topo.Server, routes map[string]string) error { + rules, err := topotools.GetKeyspaceRoutingRules(ctx, ts) + if err != nil { + return err + } + if rules == nil { + rules = make(map[string]string) + } + for fromKeyspace, toKeyspace := range routes { + rules[fromKeyspace] = toKeyspace + } + if err := topotools.SaveKeyspaceRoutingRules(ctx, ts, rules); err != nil { + return err + } + return nil +} + +func validateTenantId(dataType querypb.Type, value string) error { + switch dataType { + case querypb.Type_INT64: + _, err := strconv.Atoi(value) + if err != nil { + return fmt.Errorf("value %s is not a valid int", value) + } + case querypb.Type_VARCHAR: + // no validation needed + default: + return fmt.Errorf("unsupported data type: %s", dataType) + } + return nil +} diff --git a/go/vt/vtgate/vindexes/vschema.go b/go/vt/vtgate/vindexes/vschema.go index 8559cabc447..1762b42fa8e 100644 --- a/go/vt/vtgate/vindexes/vschema.go +++ b/go/vt/vtgate/vindexes/vschema.go @@ -25,6 +25,8 @@ import ( "strings" "time" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/json2" "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/sqlescape" @@ -67,10 +69,11 @@ type VSchema struct { // table is uniquely named, the value will be the qualified Table object // with the keyspace where this table exists. If multiple keyspaces have a // table with the same name, the value will be a `nil`. - globalTables map[string]*Table - uniqueVindexes map[string]Vindex - Keyspaces map[string]*KeyspaceSchema `json:"keyspaces"` - ShardRoutingRules map[string]string `json:"shard_routing_rules"` + globalTables map[string]*Table + uniqueVindexes map[string]Vindex + Keyspaces map[string]*KeyspaceSchema `json:"keyspaces"` + ShardRoutingRules map[string]string `json:"shard_routing_rules"` + KeyspaceRoutingRules map[string]string `json:"keyspace_routing_rules"` // created is the time when the VSchema object was created. Used to detect if a cached // copy of the vschema is stale. created time.Time @@ -229,21 +232,23 @@ func (col *Column) ToEvalengineType(collationEnv *collations.Environment) evalen // KeyspaceSchema contains the schema(table) for a keyspace. type KeyspaceSchema struct { - Keyspace *Keyspace - ForeignKeyMode vschemapb.Keyspace_ForeignKeyMode - Tables map[string]*Table - Vindexes map[string]Vindex - Views map[string]sqlparser.SelectStatement - Error error + Keyspace *Keyspace + ForeignKeyMode vschemapb.Keyspace_ForeignKeyMode + Tables map[string]*Table + Vindexes map[string]Vindex + Views map[string]sqlparser.SelectStatement + Error error + MultiTenantSpec *vschemapb.MultiTenantSpec } type ksJSON struct { - Sharded bool `json:"sharded,omitempty"` - ForeignKeyMode string `json:"foreignKeyMode,omitempty"` - Tables map[string]*Table `json:"tables,omitempty"` - Vindexes map[string]Vindex `json:"vindexes,omitempty"` - Views map[string]string `json:"views,omitempty"` - Error string `json:"error,omitempty"` + Sharded bool `json:"sharded,omitempty"` + ForeignKeyMode string `json:"foreignKeyMode,omitempty"` + Tables map[string]*Table `json:"tables,omitempty"` + Vindexes map[string]Vindex `json:"vindexes,omitempty"` + Views map[string]string `json:"views,omitempty"` + Error string `json:"error,omitempty"` + MultiTenantSpec *vschemapb.MultiTenantSpec `json:"multi_tenant_spec,omitempty"` } // findTable looks for the table with the requested tablename in the keyspace. @@ -272,10 +277,11 @@ func (ks *KeyspaceSchema) findTable( // MarshalJSON returns a JSON representation of KeyspaceSchema. func (ks *KeyspaceSchema) MarshalJSON() ([]byte, error) { ksJ := ksJSON{ - Sharded: ks.Keyspace.Sharded, - Tables: ks.Tables, - ForeignKeyMode: ks.ForeignKeyMode.String(), - Vindexes: ks.Vindexes, + Sharded: ks.Keyspace.Sharded, + Tables: ks.Tables, + ForeignKeyMode: ks.ForeignKeyMode.String(), + Vindexes: ks.Vindexes, + MultiTenantSpec: ks.MultiTenantSpec, } if ks.Error != nil { ksJ.Error = ks.Error.Error() @@ -322,6 +328,7 @@ func BuildVSchema(source *vschemapb.SrvVSchema, parser *sqlparser.Parser) (vsche buildReferences(source, vschema) buildRoutingRule(source, vschema, parser) buildShardRoutingRule(source, vschema) + buildKeyspaceRoutingRule(source, vschema) // Resolve auto-increments after routing rules are built since sequence tables also obey routing rules. resolveAutoIncrement(source, vschema, parser) return vschema @@ -362,9 +369,10 @@ func buildKeyspaces(source *vschemapb.SrvVSchema, vschema *VSchema, parser *sqlp Name: ksname, Sharded: ks.Sharded, }, - ForeignKeyMode: replaceUnspecifiedForeignKeyMode(ks.ForeignKeyMode), - Tables: make(map[string]*Table), - Vindexes: make(map[string]Vindex), + ForeignKeyMode: replaceUnspecifiedForeignKeyMode(ks.ForeignKeyMode), + Tables: make(map[string]*Table), + Vindexes: make(map[string]Vindex), + MultiTenantSpec: ks.MultiTenantSpec, } vschema.Keyspaces[ksname] = ksvschema ksvschema.Error = buildTables(ks, vschema, ksvschema, parser) @@ -981,6 +989,15 @@ func buildShardRoutingRule(source *vschemapb.SrvVSchema, vschema *VSchema) { } } +func buildKeyspaceRoutingRule(source *vschemapb.SrvVSchema, vschema *VSchema) { + vschema.KeyspaceRoutingRules = nil + if len(source.GetKeyspaceRoutingRules().GetRules()) == 0 { + return + } + rulesMap := topotools.GetKeyspaceRoutingRulesMap(source.KeyspaceRoutingRules) + vschema.KeyspaceRoutingRules = rulesMap +} + // FindTable returns a pointer to the Table. If a keyspace is specified, only tables // from that keyspace are searched. If the specified keyspace is unsharded // and no tables matched, it's considered valid: FindTable will construct a table @@ -1093,6 +1110,11 @@ func (vschema *VSchema) FirstKeyspace() *Keyspace { // FindRoutedTable finds a table checking the routing rules. func (vschema *VSchema) FindRoutedTable(keyspace, tablename string, tabletType topodatapb.TabletType) (*Table, error) { + routedKeyspace, ok := vschema.KeyspaceRoutingRules[keyspace] + if ok { + keyspace = routedKeyspace + } + qualified := tablename if keyspace != "" { qualified = keyspace + "." + tablename diff --git a/go/vt/vtgate/vindexes/vschema_test.go b/go/vt/vtgate/vindexes/vschema_test.go index ad892a66ccb..83cf8f52b05 100644 --- a/go/vt/vtgate/vindexes/vschema_test.go +++ b/go/vt/vtgate/vindexes/vschema_test.go @@ -251,6 +251,54 @@ func TestUnshardedVSchemaValid(t *testing.T) { require.NoError(t, err) } +// TestMultiTenantAttribute verifies that the MultiTenantSpec attribute is updated in KeyspaceSchema. +func TestMultiTenantAttribute(t *testing.T) { + tests := []struct { + name string + multiTenantSpec *vschemapb.MultiTenantSpec + wanted *vschemapb.MultiTenantSpec + }{ + { + name: "Not Set", + }, { + name: "Empty MultiTenantSpec", + multiTenantSpec: &vschemapb.MultiTenantSpec{ + TenantIdColumnName: "", + }, + wanted: &vschemapb.MultiTenantSpec{}, + }, { + name: "String", + multiTenantSpec: &vschemapb.MultiTenantSpec{ + TenantIdColumnName: "tenant_id", + TenantIdColumnType: querypb.Type_VARCHAR, + }, + wanted: &vschemapb.MultiTenantSpec{ + TenantIdColumnName: "tenant_id", + TenantIdColumnType: querypb.Type_VARCHAR, + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ksSchema, err := BuildKeyspace(&vschemapb.Keyspace{ + Sharded: false, + MultiTenantSpec: test.multiTenantSpec, + Vindexes: make(map[string]*vschemapb.Vindex), + Tables: make(map[string]*vschemapb.Table), + }, sqlparser.NewTestParser()) + require.NoError(t, err) + if test.multiTenantSpec == nil { + require.Empty(t, test.wanted) + } else { + require.NotNil(t, test.wanted) + require.Equal(t, test.wanted.TenantIdColumnName, ksSchema.MultiTenantSpec.TenantIdColumnName) + require.Equal(t, test.wanted.TenantIdColumnType, ksSchema.MultiTenantSpec.TenantIdColumnType) + } + + }) + } +} + func TestForeignKeyMode(t *testing.T) { tests := []struct { name string @@ -2609,6 +2657,11 @@ func TestVSchemaPBJSON(t *testing.T) { in := ` { "sharded": true, + "foreignKeyMode": "unmanaged", + "multi_tenant_spec": { + "tenant_id_column_name": "tenant_id", + "tenant_id_column_type": 265 + }, "tables": { "t1": { "column_vindexes":[{ @@ -2637,7 +2690,12 @@ func TestVSchemaPBJSON(t *testing.T) { t.Error(err) } want := vschemapb.Keyspace{ - Sharded: true, + Sharded: true, + ForeignKeyMode: vschemapb.Keyspace_unmanaged, + MultiTenantSpec: &vschemapb.MultiTenantSpec{ + TenantIdColumnName: "tenant_id", + TenantIdColumnType: querypb.Type_INT64, + }, Tables: map[string]*vschemapb.Table{ "t1": { ColumnVindexes: []*vschemapb.ColumnVindex{ @@ -2682,6 +2740,10 @@ func TestVSchemaJSON(t *testing.T) { in := map[string]*KeyspaceSchema{ "unsharded": { ForeignKeyMode: vschemapb.Keyspace_managed, + MultiTenantSpec: &vschemapb.MultiTenantSpec{ + TenantIdColumnName: "tenant_id", + TenantIdColumnType: querypb.Type_INT64, + }, Keyspace: &Keyspace{ Name: "k1", }, @@ -2775,6 +2837,10 @@ func TestVSchemaJSON(t *testing.T) { "type": "sequence", "name": "n2" } + }, + "multi_tenant_spec": { + "tenant_id_column_name": "tenant_id", + "tenant_id_column_type": 265 } } }` diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication.go b/go/vt/vttablet/tabletmanager/rpc_vreplication.go index de4a9c21c8d..9e1cf1a2471 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication.go @@ -43,13 +43,13 @@ import ( const ( // Create a new VReplication workflow record. - sqlCreateVReplicationWorkflow = "insert into %s.vreplication (workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) values (%a, %a, '', 0, 0, %a, %a, now(), 0, %a, %a, %a, %a, %a)" + sqlCreateVReplicationWorkflow = "insert into %s.vreplication (workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) values (%a, %a, '', 0, 0, %a, %a, now(), 0, %a, %a, %a, %a, %a, %a)" sqlHasVReplicationWorkflows = "select if(count(*) > 0, 1, 0) as has_workflows from %s.vreplication where db_name = %a" // Read all VReplication workflows. The final format specifier is used to // optionally add any additional predicates to the query. - sqlReadVReplicationWorkflows = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from %s.vreplication where db_name = %a%s group by workflow, id order by workflow, id" + sqlReadVReplicationWorkflows = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from %s.vreplication where db_name = %a%s group by workflow, id order by workflow, id" // Read a VReplication workflow. - sqlReadVReplicationWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from %s.vreplication where workflow = %a and db_name = %a" + sqlReadVReplicationWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from %s.vreplication where workflow = %a and db_name = %a" // Delete VReplication records for the given workflow. sqlDeleteVReplicationWorkflow = "delete from %s.vreplication where workflow = %a and db_name = %a" // Retrieve the current configuration values for a workflow's vreplication stream(s). @@ -96,9 +96,11 @@ func (tm *TabletManager) CreateVReplicationWorkflow(ctx context.Context, req *ta "workflowType": sqltypes.Int64BindVariable(int64(req.WorkflowType)), "workflowSubType": sqltypes.Int64BindVariable(int64(req.WorkflowSubType)), "deferSecondaryKeys": sqltypes.BoolBindVariable(req.DeferSecondaryKeys), + "options": sqltypes.StringBindVariable(req.Options), } parsed := sqlparser.BuildParsedQuery(sqlCreateVReplicationWorkflow, sidecar.GetIdentifier(), - ":workflow", ":source", ":cells", ":tabletTypes", ":state", ":dbname", ":workflowType", ":workflowSubType", ":deferSecondaryKeys", + ":workflow", ":source", ":cells", ":tabletTypes", ":state", ":dbname", ":workflowType", ":workflowSubType", + ":deferSecondaryKeys", ":options", ) stmt, err := parsed.GenerateQuery(bindVars, nil) if err != nil { @@ -212,7 +214,7 @@ func (tm *TabletManager) ReadVReplicationWorkflows(ctx context.Context, req *tab } workflows[workflow].WorkflowSubType = binlogdatapb.VReplicationWorkflowSubType(wfst) workflows[workflow].DeferSecondaryKeys = row["defer_secondary_keys"].ToString() == "1" - + workflows[workflow].Options = row["options"].ToString() // Now the individual streams (there can be more than 1 with shard merges). if workflows[workflow].Streams == nil { workflows[workflow].Streams = make([]*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream, 0, 1) @@ -320,7 +322,7 @@ func (tm *TabletManager) ReadVReplicationWorkflow(ctx context.Context, req *tabl } resp.WorkflowSubType = binlogdatapb.VReplicationWorkflowSubType(wfst) resp.DeferSecondaryKeys = rows[0]["defer_secondary_keys"].ToString() == "1" - + resp.Options = rows[0]["options"].ToString() // Now the individual streams (there can be more than 1 with shard merges). for i, row := range rows { streams[i] = &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{} diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go index 2190534247e..42e5129b40e 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go @@ -54,7 +54,7 @@ import ( ) const ( - insertVReplicationPrefix = "insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys)" + insertVReplicationPrefix = "insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options)" checkForJournal = "/select val from _vt.resharding_journal where id=" getWorkflowState = "select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=%d" getCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" @@ -74,9 +74,9 @@ const ( updatePickedSourceTablet = `update _vt.vreplication set message='Picked source tablet: cell:\"%s\" uid:%d' where id=%d` getRowsCopied = "SELECT rows_copied FROM _vt.vreplication WHERE id=%d" hasWorkflows = "select if(count(*) > 0, 1, 0) as has_workflows from _vt.vreplication where db_name = '%s'" - readAllWorkflows = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = '%s'%s group by workflow, id order by workflow, id" - readWorkflowsLimited = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = '%s' and workflow in ('%s') group by workflow, id order by workflow, id" - readWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where workflow = '%s' and db_name = '%s'" + readAllWorkflows = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where db_name = '%s'%s group by workflow, id order by workflow, id" + readWorkflowsLimited = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where db_name = '%s' and workflow in ('%s') group by workflow, id order by workflow, id" + readWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where workflow = '%s' and db_name = '%s'" readWorkflowConfig = "select id, source, cell, tablet_types, state, message from _vt.vreplication where workflow = '%s'" updateWorkflow = "update _vt.vreplication set state = '%s', source = '%s', cell = '%s', tablet_types = '%s' where id in (%d)" ) @@ -135,7 +135,7 @@ func TestCreateVReplicationWorkflow(t *testing.T) { Cells: tenv.cells, AllTables: true, }, - query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"}}', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 0)`, + query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"}}', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 0, '{}')`, insertVReplicationPrefix, wf, sourceKs, shard, tenv.cells[0], tenv.dbName), }, { @@ -170,7 +170,7 @@ func TestCreateVReplicationWorkflow(t *testing.T) { DeferSecondaryKeys: true, AutoStart: true, }, - query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"}} on_ddl:EXEC stop_after_copy:true source_time_zone:\"EDT\" target_time_zone:\"UTC\"', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 1)`, + query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"}} on_ddl:EXEC stop_after_copy:true source_time_zone:\"EDT\" target_time_zone:\"UTC\"', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 1, '{}')`, insertVReplicationPrefix, wf, sourceKs, shard, tenv.cells[0], tenv.dbName), }, { @@ -210,7 +210,7 @@ func TestCreateVReplicationWorkflow(t *testing.T) { DeferSecondaryKeys: true, AutoStart: true, }, - query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"wut\" filter:\"select * from wut\"} rules:{match:\"zt\" filter:\"select * from zt\"}} on_ddl:EXEC stop_after_copy:true source_time_zone:\"EDT\" target_time_zone:\"UTC\"', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 1)`, + query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"wut\" filter:\"select * from wut\"} rules:{match:\"zt\" filter:\"select * from zt\"}} on_ddl:EXEC stop_after_copy:true source_time_zone:\"EDT\" target_time_zone:\"UTC\"', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 1, '{}')`, insertVReplicationPrefix, wf, sourceKs, shard, tenv.cells[0], tenv.dbName), }, { @@ -250,7 +250,7 @@ func TestCreateVReplicationWorkflow(t *testing.T) { DeferSecondaryKeys: true, AutoStart: true, }, - query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"wut\" filter:\"select * from wut\"} rules:{match:\"zt\" filter:\"select * from zt\"}} on_ddl:EXEC stop_after_copy:true source_time_zone:\"EDT\" target_time_zone:\"UTC\"', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 1)`, + query: fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"wut\" filter:\"select * from wut\"} rules:{match:\"zt\" filter:\"select * from zt\"}} on_ddl:EXEC stop_after_copy:true source_time_zone:\"EDT\" target_time_zone:\"UTC\"', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 1, '{}')`, insertVReplicationPrefix, wf, sourceKs, shard, tenv.cells[0], tenv.dbName), }, } @@ -393,7 +393,7 @@ func TestMoveTables(t *testing.T) { tenv.tmc.setVReplicationExecResults(ftc.tablet, getCopyState, &sqltypes.Result{}) ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) - insert := fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', %d, 0, 0)`, + insert := fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', %d, 0, 0, '{}')`, insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, ftc.tablet.Shard, tenv.cells[0], tenv.dbName, vreplID) ftc.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: 1}, nil) ftc.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) @@ -993,7 +993,7 @@ func TestSourceShardSelection(t *testing.T) { } tt.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) tt.vrdbClient.ExpectRequest( - fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 0)`, + fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 0, '{}')`, insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, tt.tablet.Shard, tenv.cells[0], tenv.dbName), &sqltypes.Result{InsertID: uint64(i + 1)}, err, @@ -1077,7 +1077,7 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) { targetTablet.vrdbClient.ExpectRequest( fmt.Sprintf("%s %s", insertVReplicationPrefix, - fmt.Sprintf(`values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"%s\" filter:\"select * from %s\"}} source_time_zone:\"%s\" target_time_zone:\"UTC\"', '', 0, 0, '%s', 'primary', now(), 0, 'Stopped', '%s', 1, 0, 0)`, + fmt.Sprintf(`values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"%s\" filter:\"select * from %s\"}} source_time_zone:\"%s\" target_time_zone:\"UTC\"', '', 0, 0, '%s', 'primary', now(), 0, 'Stopped', '%s', 1, 0, 0, '{}')`, wf, sourceKs, shard, table, table, invalidTimeZone, strings.Join(tenv.cells, ","), tenv.dbName), ), &sqltypes.Result{ @@ -1753,7 +1753,7 @@ func TestMaterializerOneToOne(t *testing.T) { // the test with an error as at this point we've tested what // we wanted to test. insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"} rules:{match:\"t4\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"} rules:{match:\"t4\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, shard, tenv.cells[0], tenv.dbName) targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) @@ -1818,7 +1818,7 @@ func TestMaterializerManyToOne(t *testing.T) { bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}", sourceKs, sourceShard) insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, sourceShard, tenv.cells[0], tenv.dbName) if vreplID == 1 { targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) @@ -1914,7 +1914,7 @@ func TestMaterializerOneToMany(t *testing.T) { bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, '%s.xxhash', '%s')\"}}", sourceKs, sourceShard, targetKs, targetShard) insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) if targetShard == "-80" { targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) @@ -2014,7 +2014,7 @@ func TestMaterializerManyToMany(t *testing.T) { bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, '%s.xxhash', '%s')\"}}", sourceKs, sourceShard, targetKs, targetShard) insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) if targetShard == "80-" && sourceShard == "40-" { // Last insert targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) @@ -2115,7 +2115,7 @@ func TestMaterializerMulticolumnVindex(t *testing.T) { bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, c2, '%s.region', '%s')\"}}", sourceKs, sourceShard, targetKs, targetShard) insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) if targetShard == "-80" { targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) @@ -2199,7 +2199,7 @@ func TestMaterializerDeploySchema(t *testing.T) { // the test with an error as at this point we've tested what // we wanted to test. insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, shard, tenv.cells[0], tenv.dbName) targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) @@ -2269,7 +2269,7 @@ func TestMaterializerCopySchema(t *testing.T) { // the test with an error as at this point we've tested what // we wanted to test. insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, shard, tenv.cells[0], tenv.dbName) targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) @@ -2355,7 +2355,7 @@ func TestMaterializerExplicitColumns(t *testing.T) { bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, '%s.region', '%s')\"}}", sourceKs, sourceShard, targetKs, targetShard) insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) if targetShard == "-80" { targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) @@ -2455,7 +2455,7 @@ func TestMaterializerRenamedColumns(t *testing.T) { bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, '%s.region', '%s')\"}}", sourceKs, sourceShard, targetKs, targetShard) insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) if targetShard == "-80" { targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) @@ -2530,7 +2530,7 @@ func TestMaterializerStopAfterCopy(t *testing.T) { // the test with an error as at this point we've tested what // we wanted to test. insert := insertVReplicationPrefix + - fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}} stop_after_copy:true', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}} stop_after_copy:true', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`, wf, sourceKs, shard, tenv.cells[0], tenv.dbName) targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) @@ -3165,7 +3165,7 @@ func TestBuildReadVReplicationWorkflowsQuery(t *testing.T) { IncludeStates: []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Stopped, binlogdatapb.VReplicationWorkflowState_Error}, ExcludeFrozen: true, }, - want: "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = 'vt_testks' and message != 'FROZEN' and id in (1,2,3) and workflow in ('wf1','wf2') and workflow not in ('1wf') and state in ('Stopped','Error') group by workflow, id order by workflow, id", + want: "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where db_name = 'vt_testks' and message != 'FROZEN' and id in (1,2,3) and workflow in ('wf1','wf2') and workflow not in ('1wf') and state in ('Stopped','Error') group by workflow, id order by workflow, id", }, { name: "2 workflows if running", @@ -3173,7 +3173,7 @@ func TestBuildReadVReplicationWorkflowsQuery(t *testing.T) { IncludeWorkflows: []string{"wf1", "wf2"}, IncludeStates: []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Running}, }, - want: "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = 'vt_testks' and workflow in ('wf1','wf2') and state in ('Running') group by workflow, id order by workflow, id", + want: "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where db_name = 'vt_testks' and workflow in ('wf1','wf2') and state in ('Running') group by workflow, id order by workflow, id", }, } for _, tt := range tests { diff --git a/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go b/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go index 62af8c9396d..6a127b084b5 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go @@ -40,7 +40,7 @@ type InsertGenerator struct { // NewInsertGenerator creates a new InsertGenerator. func NewInsertGenerator(state binlogdatapb.VReplicationWorkflowState, dbname string) *InsertGenerator { buf := &strings.Builder{} - buf.WriteString("insert into _vt.vreplication(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) values ") + buf.WriteString("insert into _vt.vreplication(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) values ") return &InsertGenerator{ buf: buf, state: state.String(), @@ -53,7 +53,7 @@ func NewInsertGenerator(state binlogdatapb.VReplicationWorkflowState, dbname str func (ig *InsertGenerator) AddRow(workflow string, bls *binlogdatapb.BinlogSource, pos, cell, tabletTypes string, workflowType binlogdatapb.VReplicationWorkflowType, workflowSubType binlogdatapb.VReplicationWorkflowSubType, deferSecondaryKeys bool) { protoutil.SortBinlogSourceTables(bls) - fmt.Fprintf(ig.buf, "%s(%v, %v, %v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d, %v)", + fmt.Fprintf(ig.buf, "%s(%v, %v, %v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d, %v, %v)", ig.prefix, encodeString(workflow), encodeString(bls.String()), @@ -68,6 +68,7 @@ func (ig *InsertGenerator) AddRow(workflow string, bls *binlogdatapb.BinlogSourc workflowType, workflowSubType, deferSecondaryKeys, + "'{}'", ) ig.prefix = ", " } diff --git a/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go b/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go index 5ccdfe3da10..2b07308c4c2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go @@ -28,11 +28,11 @@ func TestInsertGenerator(t *testing.T) { ig := NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, "a") ig.now = 111 ig.AddRow("b", &binlogdatapb.BinlogSource{Keyspace: "c"}, "d", "e", "f", binlogdatapb.VReplicationWorkflowType_Materialize, binlogdatapb.VReplicationWorkflowSubType_None, false) - want := `insert into _vt.vreplication(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) values ` + - `('b', 'keyspace:\"c\"', 'd', 9223372036854775807, 9223372036854775807, 'e', 'f', 111, 0, 'Stopped', 'a', 0, 0, false)` + want := `insert into _vt.vreplication(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) values ` + + `('b', 'keyspace:\"c\"', 'd', 9223372036854775807, 9223372036854775807, 'e', 'f', 111, 0, 'Stopped', 'a', 0, 0, false, '{}')` assert.Equal(t, ig.String(), want) ig.AddRow("g", &binlogdatapb.BinlogSource{Keyspace: "h"}, "i", "j", "k", binlogdatapb.VReplicationWorkflowType_Reshard, binlogdatapb.VReplicationWorkflowSubType_Partial, true) - want += `, ('g', 'keyspace:\"h\"', 'i', 9223372036854775807, 9223372036854775807, 'j', 'k', 111, 0, 'Stopped', 'a', 4, 1, true)` + want += `, ('g', 'keyspace:\"h\"', 'i', 9223372036854775807, 9223372036854775807, 'j', 'k', 111, 0, 'Stopped', 'a', 4, 1, true, '{}')` assert.Equal(t, ig.String(), want) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go index fb5648f49af..2e7c539e840 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go @@ -614,7 +614,7 @@ func testPlayerCopyTables(t *testing.T) { expectDBClientQueries(t, qh.Expect( // Filters should be lexicographically ordered by name. - regexp.QuoteMeta("/insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type) values ('test', 'keyspace:\\\"vttest\\\" shard:\\\"0\\\" filter:{rules:{match:\\\"ast1\\\" filter:\\\"select * from ast1\\\"} rules:{match:\\\"dst1\\\" filter:\\\"select id, val, val as val2, d, j from src1\\\"} rules:{match:\\\"/yes\\\"}}'"), + regexp.QuoteMeta("/insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, options) values ('test', 'keyspace:\\\"vttest\\\" shard:\\\"0\\\" filter:{rules:{match:\\\"ast1\\\" filter:\\\"select * from ast1\\\"} rules:{match:\\\"dst1\\\" filter:\\\"select id, val, val as val2, d, j from src1\\\"} rules:{match:\\\"/yes\\\"}}'"), "/update _vt.vreplication set message='Picked source tablet.*", // Create the list of tables to copy and transition to Copying state. "begin", diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go index ac0691dc8bc..9796f7ef77c 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go @@ -1817,7 +1817,7 @@ func TestPlayerDDL(t *testing.T) { func TestGTIDCompress(t *testing.T) { ctx := context.Background() defer deleteTablet(addTablet(100)) - err := env.Mysqld.ExecuteSuperQuery(ctx, "insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state,db_name) values (1, '', '', '', 0,0,0,0,'Stopped','')") + err := env.Mysqld.ExecuteSuperQuery(ctx, "insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state,db_name, options) values (1, '', '', '', 0,0,0,0,'Stopped','', '{}')") require.NoError(t, err) type testCase struct { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go index 3be0525dc88..355df8fdd66 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go @@ -227,7 +227,7 @@ func TestDeferSecondaryKeys(t *testing.T) { defer dbClient.Close() dbName := dbClient.DBName() // Ensure there's a dummy vreplication workflow record - _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s') on duplicate key update workflow='test', source='', pos='', max_tps=99999, max_replication_lag=99999, time_updated=0, transaction_timestamp=0, state='Running', db_name='%s'", + _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, options) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s', '{}') on duplicate key update workflow='test', source='', pos='', max_tps=99999, max_replication_lag=99999, time_updated=0, transaction_timestamp=0, state='Running', db_name='%s'", id, dbName, dbName), 1) require.NoError(t, err) defer func() { @@ -383,7 +383,7 @@ func TestDeferSecondaryKeys(t *testing.T) { postStashHook: func() error { myid := id + 1000 // Insert second vreplication record to simulate a second controller/vreplicator - _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s')", + _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, options) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s', '{}')", myid, dbName), 1) if err != nil { return err @@ -617,7 +617,7 @@ func TestCancelledDeferSecondaryKeys(t *testing.T) { defer dbClient.Close() dbName := dbClient.DBName() // Ensure there's a dummy vreplication workflow record - _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s') on duplicate key update workflow='test', source='', pos='', max_tps=99999, max_replication_lag=99999, time_updated=0, transaction_timestamp=0, state='Running', db_name='%s'", + _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, options) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s', '{}') on duplicate key update workflow='test', source='', pos='', max_tps=99999, max_replication_lag=99999, time_updated=0, transaction_timestamp=0, state='Running', db_name='%s'", id, dbName, dbName), 1) require.NoError(t, err) defer func() { @@ -736,7 +736,7 @@ func TestResumingFromPreviousWorkflowKeepingRowsCopied(t *testing.T) { dbName := dbClient.DBName() rowsCopied := int64(500000) // Ensure there's an existing vreplication workflow - _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, rows_copied) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s', %v) on duplicate key update workflow='test', source='', pos='', max_tps=99999, max_replication_lag=99999, time_updated=0, transaction_timestamp=0, state='Running', db_name='%s', rows_copied=%v", + _, err = dbClient.ExecuteFetch(fmt.Sprintf("insert into _vt.vreplication (id, workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, rows_copied, options) values (%d, 'test', '', '', 99999, 99999, 0, 0, 'Running', '%s', %v, '{}') on duplicate key update workflow='test', source='', pos='', max_tps=99999, max_replication_lag=99999, time_updated=0, transaction_timestamp=0, state='Running', db_name='%s', rows_copied=%v", id, dbName, rowsCopied, dbName, rowsCopied), 1) require.NoError(t, err) defer func() { diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go index 35bea172cd0..05f21fca271 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go @@ -141,7 +141,8 @@ func TestUpdateVSchema(t *testing.T) { } } }, - "shard_routing_rules": null + "shard_routing_rules": null, + "keyspace_routing_rules": null }` b, err := json.MarshalIndent(engine.vschema(), "", " ") if err != nil { diff --git a/go/vt/wrangler/materializer_env_test.go b/go/vt/wrangler/materializer_env_test.go index e3a3364c8fd..4de02b5abcc 100644 --- a/go/vt/wrangler/materializer_env_test.go +++ b/go/vt/wrangler/materializer_env_test.go @@ -19,12 +19,15 @@ package wrangler import ( "context" "fmt" + "os" "regexp" "strconv" "strings" "sync" "testing" + _flag "vitess.io/vitess/go/internal/flag" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/test/utils" "vitess.io/vitess/go/vt/logutil" @@ -299,3 +302,8 @@ func (tmc *testMaterializerTMClient) ApplySchema(ctx context.Context, tablet *to return nil, nil } + +func TestMain(m *testing.M) { + _flag.ParseFlagsForTest() + os.Exit(m.Run()) +} diff --git a/go/vt/wrangler/materializer_test.go b/go/vt/wrangler/materializer_test.go index e7bb66f1003..23cae954b83 100644 --- a/go/vt/wrangler/materializer_test.go +++ b/go/vt/wrangler/materializer_test.go @@ -1935,7 +1935,7 @@ func TestMaterializerOneToOne(t *testing.T) { `rules:{match:\\"t2\\" filter:\\"select.*t3\\"} `+ `rules:{match:\\"t4\\"}`+ `}', `)+ - `'', [0-9]*, [0-9]*, 'zone1', 'primary,rdonly', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false`+ + `'', [0-9]*, [0-9]*, 'zone1', 'primary,rdonly', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false, '{}'`+ `\)`+eol, &sqltypes.Result{}, ) @@ -1968,9 +1968,9 @@ func TestMaterializerManyToOne(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"-80\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ + `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"-80\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false, '{}'\)`+ `, `+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"80-\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ + `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"80-\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -2181,7 +2181,7 @@ func TestMaterializerDeploySchema(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ + `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -2219,7 +2219,7 @@ func TestMaterializerCopySchema(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ + `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) diff --git a/go/vt/wrangler/resharder_test.go b/go/vt/wrangler/resharder_test.go index 8d5a90092f3..b4a939775ca 100644 --- a/go/vt/wrangler/resharder_test.go +++ b/go/vt/wrangler/resharder_test.go @@ -34,7 +34,7 @@ import ( ) const rsSelectFrozenQuery = "select 1 from _vt.vreplication where db_name='vt_ks' and message='FROZEN' and workflow_sub_type != 1" -const insertPrefix = `/insert into _vt.vreplication\(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys\) values ` +const insertPrefix = `/insert into _vt.vreplication\(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options\) values ` const eol = "$" func TestResharderOneToMany(t *testing.T) { @@ -94,14 +94,14 @@ func TestResharderOneToMany(t *testing.T) { 200, insertPrefix+ `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '`+ - tc.cells+`', '`+tc.tabletTypes+`', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+eol, + tc.cells+`', '`+tc.tabletTypes+`', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '`+ - tc.cells+`', '`+tc.tabletTypes+`', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+eol, + tc.cells+`', '`+tc.tabletTypes+`', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery(200, "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state='Running' where db_name='vt_ks'", &sqltypes.Result{}) @@ -137,8 +137,8 @@ func TestResharderManyToOne(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"-80\\" filter:{rules:{match:\\"/.*\\" filter:\\"-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\).*`+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"80-\\" filter:{rules:{match:\\"/.*\\" filter:\\"-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"-80\\" filter:{rules:{match:\\"/.*\\" filter:\\"-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\).*`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"80-\\" filter:{rules:{match:\\"/.*\\" filter:\\"-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -172,15 +172,15 @@ func TestResharderManyToMany(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"-40\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\).*`+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"40-\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"-40\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\).*`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"40-\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"40-\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"40-\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -228,14 +228,14 @@ func TestResharderOneRefTable(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -283,14 +283,14 @@ func TestReshardStopFlags(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"-80\\"}} stop_after_copy:true', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"-80\\"}} stop_after_copy:true', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"80-\\"}} stop_after_copy:true', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"80-\\"}} stop_after_copy:true', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -347,18 +347,18 @@ func TestResharderOneRefStream(t *testing.T) { ) env.tmc.expectVRQuery(100, fmt.Sprintf("select workflow, source, cell, tablet_types from _vt.vreplication where db_name='vt_%s' and message != 'FROZEN'", env.keyspace), result) - refRow := `\('t1', 'keyspace:\\"ks1\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\"}}', '', [0-9]*, [0-9]*, 'cell1', 'primary,replica', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)` + refRow := `\('t1', 'keyspace:\\"ks1\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\"}}', '', [0-9]*, [0-9]*, 'cell1', 'primary,replica', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)` env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\).*`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\).*`+ refRow+eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\).*`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"exclude\\"} rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\).*`+ refRow+eol, &sqltypes.Result{}, ) @@ -430,14 +430,14 @@ func TestResharderNoRefStream(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) @@ -472,14 +472,14 @@ func TestResharderCopySchema(t *testing.T) { env.tmc.expectVRQuery( 200, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"-80\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) env.tmc.expectVRQuery( 210, insertPrefix+ - `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false\)`+ + `\('resharderTest', 'keyspace:\\"ks\\" shard:\\"0\\" filter:{rules:{match:\\"/.*\\" filter:\\"80-\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_ks', 4, 0, false, '{}'\)`+ eol, &sqltypes.Result{}, ) diff --git a/go/vt/wrangler/workflow_test.go b/go/vt/wrangler/workflow_test.go index 978883d5457..4f508766330 100644 --- a/go/vt/wrangler/workflow_test.go +++ b/go/vt/wrangler/workflow_test.go @@ -750,7 +750,7 @@ func expectReshardQueries(t *testing.T, tme *testShardMigraterEnv, params *VRepl dbclient.addInvariant("delete from _vt.vreplication where id in (1)", noResult) dbclient.addInvariant("delete from _vt.copy_state where vrepl_id in (1)", noResult) dbclient.addInvariant("delete from _vt.post_copy_action where vrepl_id in (1)", noResult) - dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type)", &sqltypes.Result{InsertID: uint64(1)}) + dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, options)", &sqltypes.Result{InsertID: uint64(1)}) dbclient.addInvariant("select id from _vt.vreplication where id = 1", resultid1) dbclient.addInvariant("select id from _vt.vreplication where id = 2", resultid2) dbclient.addInvariant("select * from _vt.vreplication where id = 1", runningResult(1)) @@ -818,7 +818,7 @@ func expectMoveTablesQueries(t *testing.T, tme *testMigraterEnv, params *VReplic dbclient.addInvariant("select id from _vt.vreplication where id = 2", resultid2) dbclient.addInvariant("update _vt.vreplication set state = 'Stopped', message = 'stopped for cutover' where id in (1)", noResult) dbclient.addInvariant("update _vt.vreplication set state = 'Stopped', message = 'stopped for cutover' where id in (2)", noResult) - dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type)", &sqltypes.Result{InsertID: uint64(1)}) + dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, options)", &sqltypes.Result{InsertID: uint64(1)}) dbclient.addInvariant("update _vt.vreplication set message = 'FROZEN'", noResult) dbclient.addInvariant("select 1 from _vt.vreplication where db_name='vt_ks2' and workflow='test' and message!='FROZEN'", noResult) dbclient.addInvariant("delete from _vt.vreplication where id in (1)", noResult) @@ -837,7 +837,7 @@ func expectMoveTablesQueries(t *testing.T, tme *testMigraterEnv, params *VReplic for _, dbclient := range tme.dbSourceClients { dbclient.addInvariant("select val from _vt.resharding_journal", noResult) dbclient.addInvariant("update _vt.vreplication set message = 'FROZEN'", noResult) - dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type)", &sqltypes.Result{InsertID: uint64(1)}) + dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, options)", &sqltypes.Result{InsertID: uint64(1)}) dbclient.addInvariant("update _vt.vreplication set state = 'Stopped', message = 'stopped for cutover' where id in (1)", noResult) dbclient.addInvariant("update _vt.vreplication set state = 'Stopped', message = 'stopped for cutover' where id in (2)", noResult) dbclient.addInvariant("select id from _vt.vreplication where id = 1", resultid1) @@ -846,7 +846,7 @@ func expectMoveTablesQueries(t *testing.T, tme *testMigraterEnv, params *VReplic dbclient.addInvariant("delete from _vt.vreplication where id in (1)", noResult) dbclient.addInvariant("delete from _vt.copy_state where vrepl_id in (1)", noResult) dbclient.addInvariant("delete from _vt.post_copy_action where vrepl_id in (1)", noResult) - dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type)", &sqltypes.Result{InsertID: uint64(1)}) + dbclient.addInvariant("insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, options)", &sqltypes.Result{InsertID: uint64(1)}) dbclient.addInvariant("select * from _vt.vreplication where id = 1", runningResult(1)) dbclient.addInvariant("select * from _vt.vreplication where id = 2", runningResult(2)) dbclient.addInvariant("insert into _vt.resharding_journal", noResult) diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto index 2ce7e5cb1fc..4af90188a39 100644 --- a/proto/tabletmanagerdata.proto +++ b/proto/tabletmanagerdata.proto @@ -551,6 +551,7 @@ message CreateVReplicationWorkflowRequest { bool auto_start = 9; // Should the workflow stop after the copy phase. bool stop_after_copy = 10; + string options = 11; } message CreateVReplicationWorkflowResponse { @@ -616,6 +617,7 @@ message ReadVReplicationWorkflowResponse { string component_throttled = 14; } repeated Stream streams = 11; + string options = 12; } message VDiffRequest { diff --git a/proto/vschema.proto b/proto/vschema.proto index 45c1b7a3aa9..dd327f863dc 100644 --- a/proto/vschema.proto +++ b/proto/vschema.proto @@ -54,6 +54,16 @@ message Keyspace { unmanaged = 2; managed = 3; } + + // multi_tenant_mode specifies that the keyspace is multi-tenant. Currently used during migrations with MoveTables. + MultiTenantSpec multi_tenant_spec = 6; +} + +message MultiTenantSpec { + // tenant_column is the name of the column that specifies the tenant id. + string tenant_id_column_name = 1; + // tenant_column_type is the type of the column that specifies the tenant id. + query.Type tenant_id_column_type = 2; } // Vindex is the vindex info for a Keyspace. @@ -142,6 +152,7 @@ message SrvVSchema { map keyspaces = 1; RoutingRules routing_rules = 2; // table routing rules ShardRoutingRules shard_routing_rules = 3; + KeyspaceRoutingRules keyspace_routing_rules = 4; } // ShardRoutingRules specify the shard routing rules for the VSchema. @@ -149,9 +160,19 @@ message ShardRoutingRules { repeated ShardRoutingRule rules = 1; } -// RoutingRule specifies a routing rule. +// ShardRoutingRule specifies a routing rule. message ShardRoutingRule { string from_keyspace = 1; string to_keyspace = 2; string shard = 3; } + +message KeyspaceRoutingRules { + repeated KeyspaceRoutingRule rules = 1; +} + +message KeyspaceRoutingRule { + string from_keyspace = 1; + string to_keyspace = 2; +} + diff --git a/proto/vtctldata.proto b/proto/vtctldata.proto index 5f6b78bdd92..3feb1d45dfd 100644 --- a/proto/vtctldata.proto +++ b/proto/vtctldata.proto @@ -97,6 +97,7 @@ message MaterializeSettings { bool defer_secondary_keys = 14; tabletmanagerdata.TabletSelectionPreference tablet_selection_preference = 15; bool atomic_copy = 16; + WorkflowOptions workflow_options = 17; } /* Data types for VtctldServer */ @@ -205,6 +206,11 @@ message Shard { topodata.Shard shard = 3; } +message WorkflowOptions { + string tenant_id = 1; + string source_keyspace_alias = 2; +} + // TODO: comment the hell out of this. message Workflow { string name = 1; @@ -222,6 +228,9 @@ message Workflow { int64 max_v_replication_transaction_lag = 8; // This specifies whether to defer the creation of secondary keys. bool defer_secondary_keys = 9; + // These are additional (optional) settings for vreplication workflows. Previously we used to add it to the + // binlogdata.BinlogSource proto object. More details in go/vt/sidecardb/schema/vreplication.sql. + WorkflowOptions options = 10; message ReplicationLocation { string keyspace = 1; @@ -307,6 +316,22 @@ message AddCellsAliasRequest { message AddCellsAliasResponse { } + +message ApplyKeyspaceRoutingRulesRequest { + vschema.KeyspaceRoutingRules keyspace_routing_rules = 1; + // SkipRebuild, if set, will cause ApplyKeyspaceRoutingRules to skip rebuilding the + // SrvVSchema objects in each cell in RebuildCells. + bool skip_rebuild = 2; + // RebuildCells limits the SrvVSchema rebuild to the specified cells. If not + // provided the SrvVSchema will be rebuilt in every cell in the topology. + // + // Ignored if SkipRebuild is set. + repeated string rebuild_cells = 3; +} + +message ApplyKeyspaceRoutingRulesResponse { +} + message ApplyRoutingRulesRequest { vschema.RoutingRules routing_rules = 1; // SkipRebuild, if set, will cause ApplyRoutingRules to skip rebuilding the @@ -337,6 +362,8 @@ message ApplyShardRoutingRulesRequest { message ApplyShardRoutingRulesResponse { } + + message ApplySchemaRequest { string keyspace = 1; reserved 2; @@ -808,6 +835,13 @@ message GetPermissionsResponse { tabletmanagerdata.Permissions permissions = 1; } +message GetKeyspaceRoutingRulesRequest { +} + +message GetKeyspaceRoutingRulesResponse { + vschema.KeyspaceRoutingRules keyspace_routing_rules = 1; +} + message GetRoutingRulesRequest { } @@ -1217,6 +1251,7 @@ message MoveTablesCreateRequest { bool no_routing_rules = 18; // Run a single copy phase for the entire database. bool atomic_copy = 19; + WorkflowOptions workflow_options = 20; } message MoveTablesCreateResponse { diff --git a/proto/vtctlservice.proto b/proto/vtctlservice.proto index 6307150f669..8abc37dab80 100644 --- a/proto/vtctlservice.proto +++ b/proto/vtctlservice.proto @@ -45,6 +45,8 @@ service Vtctld { rpc ApplyRoutingRules(vtctldata.ApplyRoutingRulesRequest) returns (vtctldata.ApplyRoutingRulesResponse) {}; // ApplySchema applies a schema to a keyspace. rpc ApplySchema(vtctldata.ApplySchemaRequest) returns (vtctldata.ApplySchemaResponse) {}; + // ApplyKeyspaceRoutingRules applies the VSchema keyspace routing rules. + rpc ApplyKeyspaceRoutingRules(vtctldata.ApplyKeyspaceRoutingRulesRequest) returns (vtctldata.ApplyKeyspaceRoutingRulesResponse) {}; // ApplyShardRoutingRules applies the VSchema shard routing rules. rpc ApplyShardRoutingRules(vtctldata.ApplyShardRoutingRulesRequest) returns (vtctldata.ApplyShardRoutingRulesResponse) {}; // ApplyVSchema applies a vschema to a keyspace. @@ -123,6 +125,8 @@ service Vtctld { rpc GetKeyspace(vtctldata.GetKeyspaceRequest) returns (vtctldata.GetKeyspaceResponse) {}; // GetKeyspaces returns the keyspace struct of all keyspaces in the topo. rpc GetKeyspaces(vtctldata.GetKeyspacesRequest) returns (vtctldata.GetKeyspacesResponse) {}; + // GetKeyspaceRoutingRules returns the VSchema keyspace routing rules. + rpc GetKeyspaceRoutingRules(vtctldata.GetKeyspaceRoutingRulesRequest) returns (vtctldata.GetKeyspaceRoutingRulesResponse) {}; // GetPermissions returns the permissions set on the remote tablet. rpc GetPermissions(vtctldata.GetPermissionsRequest) returns (vtctldata.GetPermissionsResponse) {}; // GetRoutingRules returns the VSchema routing rules. diff --git a/test/ci_workflow_gen.go b/test/ci_workflow_gen.go index d047d588f2a..fd890df6c32 100644 --- a/test/ci_workflow_gen.go +++ b/test/ci_workflow_gen.go @@ -118,6 +118,7 @@ var ( "vreplication_partial_movetables_and_materialize", "vreplication_foreign_key_stress", "vreplication_migrate_vdiff2_convert_tz", + "vreplication_multi_tenant", "schemadiff_vrepl", "topo_connection_cache", "vtgate_partial_keyspace", diff --git a/test/config.json b/test/config.json index 9bbab938f77..0324ee54ef1 100644 --- a/test/config.json +++ b/test/config.json @@ -1031,6 +1031,15 @@ "RetryMax": 0, "Tags": [] }, + "vreplication_multi_tenant": { + "File": "unused.go", + "Args": ["vitess.io/vitess/go/test/endtoend/vreplication","-run", "MultiTenant"], + "Command": [], + "Manual": false, + "Shard": "vreplication_multi_tenant", + "RetryMax": 0, + "Tags": [] + }, "vreplication_partial_movetables_basic": { "File": "unused.go", "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "PartialMoveTablesBasic"], diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 93248bec5d3..4a33ef23fb3 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -27587,6 +27587,9 @@ export namespace tabletmanagerdata { /** CreateVReplicationWorkflowRequest stop_after_copy */ stop_after_copy?: (boolean|null); + + /** CreateVReplicationWorkflowRequest options */ + options?: (string|null); } /** Represents a CreateVReplicationWorkflowRequest. */ @@ -27628,6 +27631,9 @@ export namespace tabletmanagerdata { /** CreateVReplicationWorkflowRequest stop_after_copy. */ public stop_after_copy: boolean; + /** CreateVReplicationWorkflowRequest options. */ + public options: string; + /** * Creates a new CreateVReplicationWorkflowRequest instance using the specified properties. * @param [properties] Properties to set @@ -28538,6 +28544,9 @@ export namespace tabletmanagerdata { /** ReadVReplicationWorkflowResponse streams */ streams?: (tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream[]|null); + + /** ReadVReplicationWorkflowResponse options */ + options?: (string|null); } /** Represents a ReadVReplicationWorkflowResponse. */ @@ -28579,6 +28588,9 @@ export namespace tabletmanagerdata { /** ReadVReplicationWorkflowResponse streams. */ public streams: tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream[]; + /** ReadVReplicationWorkflowResponse options. */ + public options: string; + /** * Creates a new ReadVReplicationWorkflowResponse instance using the specified properties. * @param [properties] Properties to set @@ -42855,6 +42867,9 @@ export namespace vschema { /** Keyspace foreign_key_mode */ foreign_key_mode?: (vschema.Keyspace.ForeignKeyMode|null); + + /** Keyspace multi_tenant_spec */ + multi_tenant_spec?: (vschema.IMultiTenantSpec|null); } /** Represents a Keyspace. */ @@ -42881,6 +42896,9 @@ export namespace vschema { /** Keyspace foreign_key_mode. */ public foreign_key_mode: vschema.Keyspace.ForeignKeyMode; + /** Keyspace multi_tenant_spec. */ + public multi_tenant_spec?: (vschema.IMultiTenantSpec|null); + /** * Creates a new Keyspace instance using the specified properties. * @param [properties] Properties to set @@ -42970,6 +42988,109 @@ export namespace vschema { } } + /** Properties of a MultiTenantSpec. */ + interface IMultiTenantSpec { + + /** MultiTenantSpec tenant_id_column_name */ + tenant_id_column_name?: (string|null); + + /** MultiTenantSpec tenant_id_column_type */ + tenant_id_column_type?: (query.Type|null); + } + + /** Represents a MultiTenantSpec. */ + class MultiTenantSpec implements IMultiTenantSpec { + + /** + * Constructs a new MultiTenantSpec. + * @param [properties] Properties to set + */ + constructor(properties?: vschema.IMultiTenantSpec); + + /** MultiTenantSpec tenant_id_column_name. */ + public tenant_id_column_name: string; + + /** MultiTenantSpec tenant_id_column_type. */ + public tenant_id_column_type: query.Type; + + /** + * Creates a new MultiTenantSpec instance using the specified properties. + * @param [properties] Properties to set + * @returns MultiTenantSpec instance + */ + public static create(properties?: vschema.IMultiTenantSpec): vschema.MultiTenantSpec; + + /** + * Encodes the specified MultiTenantSpec message. Does not implicitly {@link vschema.MultiTenantSpec.verify|verify} messages. + * @param message MultiTenantSpec message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vschema.IMultiTenantSpec, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified MultiTenantSpec message, length delimited. Does not implicitly {@link vschema.MultiTenantSpec.verify|verify} messages. + * @param message MultiTenantSpec message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vschema.IMultiTenantSpec, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a MultiTenantSpec message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns MultiTenantSpec + * @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): vschema.MultiTenantSpec; + + /** + * Decodes a MultiTenantSpec message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns MultiTenantSpec + * @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)): vschema.MultiTenantSpec; + + /** + * Verifies a MultiTenantSpec 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 MultiTenantSpec message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns MultiTenantSpec + */ + public static fromObject(object: { [k: string]: any }): vschema.MultiTenantSpec; + + /** + * Creates a plain object from a MultiTenantSpec message. Also converts values to other types if specified. + * @param message MultiTenantSpec + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vschema.MultiTenantSpec, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this MultiTenantSpec to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for MultiTenantSpec + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of a Vindex. */ interface IVindex { @@ -43583,6 +43704,9 @@ export namespace vschema { /** SrvVSchema shard_routing_rules */ shard_routing_rules?: (vschema.IShardRoutingRules|null); + + /** SrvVSchema keyspace_routing_rules */ + keyspace_routing_rules?: (vschema.IKeyspaceRoutingRules|null); } /** Represents a SrvVSchema. */ @@ -43603,6 +43727,9 @@ export namespace vschema { /** SrvVSchema shard_routing_rules. */ public shard_routing_rules?: (vschema.IShardRoutingRules|null); + /** SrvVSchema keyspace_routing_rules. */ + public keyspace_routing_rules?: (vschema.IKeyspaceRoutingRules|null); + /** * Creates a new SrvVSchema instance using the specified properties. * @param [properties] Properties to set @@ -43886,6 +44013,206 @@ export namespace vschema { */ public static getTypeUrl(typeUrlPrefix?: string): string; } + + /** Properties of a KeyspaceRoutingRules. */ + interface IKeyspaceRoutingRules { + + /** KeyspaceRoutingRules rules */ + rules?: (vschema.IKeyspaceRoutingRule[]|null); + } + + /** Represents a KeyspaceRoutingRules. */ + class KeyspaceRoutingRules implements IKeyspaceRoutingRules { + + /** + * Constructs a new KeyspaceRoutingRules. + * @param [properties] Properties to set + */ + constructor(properties?: vschema.IKeyspaceRoutingRules); + + /** KeyspaceRoutingRules rules. */ + public rules: vschema.IKeyspaceRoutingRule[]; + + /** + * Creates a new KeyspaceRoutingRules instance using the specified properties. + * @param [properties] Properties to set + * @returns KeyspaceRoutingRules instance + */ + public static create(properties?: vschema.IKeyspaceRoutingRules): vschema.KeyspaceRoutingRules; + + /** + * Encodes the specified KeyspaceRoutingRules message. Does not implicitly {@link vschema.KeyspaceRoutingRules.verify|verify} messages. + * @param message KeyspaceRoutingRules message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vschema.IKeyspaceRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified KeyspaceRoutingRules message, length delimited. Does not implicitly {@link vschema.KeyspaceRoutingRules.verify|verify} messages. + * @param message KeyspaceRoutingRules message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vschema.IKeyspaceRoutingRules, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a KeyspaceRoutingRules message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns KeyspaceRoutingRules + * @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): vschema.KeyspaceRoutingRules; + + /** + * Decodes a KeyspaceRoutingRules message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns KeyspaceRoutingRules + * @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)): vschema.KeyspaceRoutingRules; + + /** + * Verifies a KeyspaceRoutingRules 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 KeyspaceRoutingRules message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns KeyspaceRoutingRules + */ + public static fromObject(object: { [k: string]: any }): vschema.KeyspaceRoutingRules; + + /** + * Creates a plain object from a KeyspaceRoutingRules message. Also converts values to other types if specified. + * @param message KeyspaceRoutingRules + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vschema.KeyspaceRoutingRules, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this KeyspaceRoutingRules to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for KeyspaceRoutingRules + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a KeyspaceRoutingRule. */ + interface IKeyspaceRoutingRule { + + /** KeyspaceRoutingRule from_keyspace */ + from_keyspace?: (string|null); + + /** KeyspaceRoutingRule to_keyspace */ + to_keyspace?: (string|null); + } + + /** Represents a KeyspaceRoutingRule. */ + class KeyspaceRoutingRule implements IKeyspaceRoutingRule { + + /** + * Constructs a new KeyspaceRoutingRule. + * @param [properties] Properties to set + */ + constructor(properties?: vschema.IKeyspaceRoutingRule); + + /** KeyspaceRoutingRule from_keyspace. */ + public from_keyspace: string; + + /** KeyspaceRoutingRule to_keyspace. */ + public to_keyspace: string; + + /** + * Creates a new KeyspaceRoutingRule instance using the specified properties. + * @param [properties] Properties to set + * @returns KeyspaceRoutingRule instance + */ + public static create(properties?: vschema.IKeyspaceRoutingRule): vschema.KeyspaceRoutingRule; + + /** + * Encodes the specified KeyspaceRoutingRule message. Does not implicitly {@link vschema.KeyspaceRoutingRule.verify|verify} messages. + * @param message KeyspaceRoutingRule message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vschema.IKeyspaceRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified KeyspaceRoutingRule message, length delimited. Does not implicitly {@link vschema.KeyspaceRoutingRule.verify|verify} messages. + * @param message KeyspaceRoutingRule message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vschema.IKeyspaceRoutingRule, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a KeyspaceRoutingRule message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns KeyspaceRoutingRule + * @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): vschema.KeyspaceRoutingRule; + + /** + * Decodes a KeyspaceRoutingRule message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns KeyspaceRoutingRule + * @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)): vschema.KeyspaceRoutingRule; + + /** + * Verifies a KeyspaceRoutingRule 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 KeyspaceRoutingRule message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns KeyspaceRoutingRule + */ + public static fromObject(object: { [k: string]: any }): vschema.KeyspaceRoutingRule; + + /** + * Creates a plain object from a KeyspaceRoutingRule message. Also converts values to other types if specified. + * @param message KeyspaceRoutingRule + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vschema.KeyspaceRoutingRule, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this KeyspaceRoutingRule to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for KeyspaceRoutingRule + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } } /** Namespace vtctldata. */ @@ -44257,6 +44584,9 @@ export namespace vtctldata { /** MaterializeSettings atomic_copy */ atomic_copy?: (boolean|null); + + /** MaterializeSettings workflow_options */ + workflow_options?: (vtctldata.IWorkflowOptions|null); } /** Represents a MaterializeSettings. */ @@ -44316,6 +44646,9 @@ export namespace vtctldata { /** MaterializeSettings atomic_copy. */ public atomic_copy: boolean; + /** MaterializeSettings workflow_options. */ + public workflow_options?: (vtctldata.IWorkflowOptions|null); + /** * Creates a new MaterializeSettings instance using the specified properties. * @param [properties] Properties to set @@ -45053,51 +45386,157 @@ export namespace vtctldata { public static getTypeUrl(typeUrlPrefix?: string): string; } - /** Properties of a Workflow. */ - interface IWorkflow { - - /** Workflow name */ - name?: (string|null); - - /** Workflow source */ - source?: (vtctldata.Workflow.IReplicationLocation|null); - - /** Workflow target */ - target?: (vtctldata.Workflow.IReplicationLocation|null); - - /** Workflow max_v_replication_lag */ - max_v_replication_lag?: (number|Long|null); + /** Properties of a WorkflowOptions. */ + interface IWorkflowOptions { - /** Workflow shard_streams */ - shard_streams?: ({ [k: string]: vtctldata.Workflow.IShardStream }|null); + /** WorkflowOptions tenant_id */ + tenant_id?: (string|null); - /** Workflow workflow_type */ - workflow_type?: (string|null); + /** WorkflowOptions source_keyspace_alias */ + source_keyspace_alias?: (string|null); + } - /** Workflow workflow_sub_type */ - workflow_sub_type?: (string|null); + /** Represents a WorkflowOptions. */ + class WorkflowOptions implements IWorkflowOptions { - /** Workflow max_v_replication_transaction_lag */ - max_v_replication_transaction_lag?: (number|Long|null); + /** + * Constructs a new WorkflowOptions. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IWorkflowOptions); - /** Workflow defer_secondary_keys */ - defer_secondary_keys?: (boolean|null); - } + /** WorkflowOptions tenant_id. */ + public tenant_id: string; - /** Represents a Workflow. */ - class Workflow implements IWorkflow { + /** WorkflowOptions source_keyspace_alias. */ + public source_keyspace_alias: string; /** - * Constructs a new Workflow. + * Creates a new WorkflowOptions instance using the specified properties. * @param [properties] Properties to set + * @returns WorkflowOptions instance */ - constructor(properties?: vtctldata.IWorkflow); + public static create(properties?: vtctldata.IWorkflowOptions): vtctldata.WorkflowOptions; - /** Workflow name. */ - public name: string; + /** + * Encodes the specified WorkflowOptions message. Does not implicitly {@link vtctldata.WorkflowOptions.verify|verify} messages. + * @param message WorkflowOptions message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IWorkflowOptions, writer?: $protobuf.Writer): $protobuf.Writer; - /** Workflow source. */ - public source?: (vtctldata.Workflow.IReplicationLocation|null); + /** + * Encodes the specified WorkflowOptions message, length delimited. Does not implicitly {@link vtctldata.WorkflowOptions.verify|verify} messages. + * @param message WorkflowOptions message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IWorkflowOptions, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a WorkflowOptions message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns WorkflowOptions + * @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): vtctldata.WorkflowOptions; + + /** + * Decodes a WorkflowOptions message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns WorkflowOptions + * @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)): vtctldata.WorkflowOptions; + + /** + * Verifies a WorkflowOptions 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 WorkflowOptions message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns WorkflowOptions + */ + public static fromObject(object: { [k: string]: any }): vtctldata.WorkflowOptions; + + /** + * Creates a plain object from a WorkflowOptions message. Also converts values to other types if specified. + * @param message WorkflowOptions + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.WorkflowOptions, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this WorkflowOptions to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for WorkflowOptions + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a Workflow. */ + interface IWorkflow { + + /** Workflow name */ + name?: (string|null); + + /** Workflow source */ + source?: (vtctldata.Workflow.IReplicationLocation|null); + + /** Workflow target */ + target?: (vtctldata.Workflow.IReplicationLocation|null); + + /** Workflow max_v_replication_lag */ + max_v_replication_lag?: (number|Long|null); + + /** Workflow shard_streams */ + shard_streams?: ({ [k: string]: vtctldata.Workflow.IShardStream }|null); + + /** Workflow workflow_type */ + workflow_type?: (string|null); + + /** Workflow workflow_sub_type */ + workflow_sub_type?: (string|null); + + /** Workflow max_v_replication_transaction_lag */ + max_v_replication_transaction_lag?: (number|Long|null); + + /** Workflow defer_secondary_keys */ + defer_secondary_keys?: (boolean|null); + + /** Workflow options */ + options?: (vtctldata.IWorkflowOptions|null); + } + + /** Represents a Workflow. */ + class Workflow implements IWorkflow { + + /** + * Constructs a new Workflow. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IWorkflow); + + /** Workflow name. */ + public name: string; + + /** Workflow source. */ + public source?: (vtctldata.Workflow.IReplicationLocation|null); /** Workflow target. */ public target?: (vtctldata.Workflow.IReplicationLocation|null); @@ -45120,6 +45559,9 @@ export namespace vtctldata { /** Workflow defer_secondary_keys. */ public defer_secondary_keys: boolean; + /** Workflow options. */ + public options?: (vtctldata.IWorkflowOptions|null); + /** * Creates a new Workflow instance using the specified properties. * @param [properties] Properties to set @@ -46366,6 +46808,206 @@ export namespace vtctldata { public static getTypeUrl(typeUrlPrefix?: string): string; } + /** Properties of an ApplyKeyspaceRoutingRulesRequest. */ + interface IApplyKeyspaceRoutingRulesRequest { + + /** ApplyKeyspaceRoutingRulesRequest keyspace_routing_rules */ + keyspace_routing_rules?: (vschema.IKeyspaceRoutingRules|null); + + /** ApplyKeyspaceRoutingRulesRequest skip_rebuild */ + skip_rebuild?: (boolean|null); + + /** ApplyKeyspaceRoutingRulesRequest rebuild_cells */ + rebuild_cells?: (string[]|null); + } + + /** Represents an ApplyKeyspaceRoutingRulesRequest. */ + class ApplyKeyspaceRoutingRulesRequest implements IApplyKeyspaceRoutingRulesRequest { + + /** + * Constructs a new ApplyKeyspaceRoutingRulesRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplyKeyspaceRoutingRulesRequest); + + /** ApplyKeyspaceRoutingRulesRequest keyspace_routing_rules. */ + public keyspace_routing_rules?: (vschema.IKeyspaceRoutingRules|null); + + /** ApplyKeyspaceRoutingRulesRequest skip_rebuild. */ + public skip_rebuild: boolean; + + /** ApplyKeyspaceRoutingRulesRequest rebuild_cells. */ + public rebuild_cells: string[]; + + /** + * Creates a new ApplyKeyspaceRoutingRulesRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplyKeyspaceRoutingRulesRequest instance + */ + public static create(properties?: vtctldata.IApplyKeyspaceRoutingRulesRequest): vtctldata.ApplyKeyspaceRoutingRulesRequest; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesRequest.verify|verify} messages. + * @param message ApplyKeyspaceRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplyKeyspaceRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesRequest.verify|verify} messages. + * @param message ApplyKeyspaceRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplyKeyspaceRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplyKeyspaceRoutingRulesRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplyKeyspaceRoutingRulesRequest + * @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): vtctldata.ApplyKeyspaceRoutingRulesRequest; + + /** + * Decodes an ApplyKeyspaceRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplyKeyspaceRoutingRulesRequest + * @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)): vtctldata.ApplyKeyspaceRoutingRulesRequest; + + /** + * Verifies an ApplyKeyspaceRoutingRulesRequest 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 an ApplyKeyspaceRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplyKeyspaceRoutingRulesRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyKeyspaceRoutingRulesRequest; + + /** + * Creates a plain object from an ApplyKeyspaceRoutingRulesRequest message. Also converts values to other types if specified. + * @param message ApplyKeyspaceRoutingRulesRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplyKeyspaceRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplyKeyspaceRoutingRulesRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplyKeyspaceRoutingRulesRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an ApplyKeyspaceRoutingRulesResponse. */ + interface IApplyKeyspaceRoutingRulesResponse { + } + + /** Represents an ApplyKeyspaceRoutingRulesResponse. */ + class ApplyKeyspaceRoutingRulesResponse implements IApplyKeyspaceRoutingRulesResponse { + + /** + * Constructs a new ApplyKeyspaceRoutingRulesResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IApplyKeyspaceRoutingRulesResponse); + + /** + * Creates a new ApplyKeyspaceRoutingRulesResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ApplyKeyspaceRoutingRulesResponse instance + */ + public static create(properties?: vtctldata.IApplyKeyspaceRoutingRulesResponse): vtctldata.ApplyKeyspaceRoutingRulesResponse; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesResponse.verify|verify} messages. + * @param message ApplyKeyspaceRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IApplyKeyspaceRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesResponse.verify|verify} messages. + * @param message ApplyKeyspaceRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IApplyKeyspaceRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an ApplyKeyspaceRoutingRulesResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ApplyKeyspaceRoutingRulesResponse + * @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): vtctldata.ApplyKeyspaceRoutingRulesResponse; + + /** + * Decodes an ApplyKeyspaceRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ApplyKeyspaceRoutingRulesResponse + * @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)): vtctldata.ApplyKeyspaceRoutingRulesResponse; + + /** + * Verifies an ApplyKeyspaceRoutingRulesResponse 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 an ApplyKeyspaceRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ApplyKeyspaceRoutingRulesResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.ApplyKeyspaceRoutingRulesResponse; + + /** + * Creates a plain object from an ApplyKeyspaceRoutingRulesResponse message. Also converts values to other types if specified. + * @param message ApplyKeyspaceRoutingRulesResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.ApplyKeyspaceRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ApplyKeyspaceRoutingRulesResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ApplyKeyspaceRoutingRulesResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of an ApplyRoutingRulesRequest. */ interface IApplyRoutingRulesRequest { @@ -53203,6 +53845,194 @@ export namespace vtctldata { public static getTypeUrl(typeUrlPrefix?: string): string; } + /** Properties of a GetKeyspaceRoutingRulesRequest. */ + interface IGetKeyspaceRoutingRulesRequest { + } + + /** Represents a GetKeyspaceRoutingRulesRequest. */ + class GetKeyspaceRoutingRulesRequest implements IGetKeyspaceRoutingRulesRequest { + + /** + * Constructs a new GetKeyspaceRoutingRulesRequest. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IGetKeyspaceRoutingRulesRequest); + + /** + * Creates a new GetKeyspaceRoutingRulesRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns GetKeyspaceRoutingRulesRequest instance + */ + public static create(properties?: vtctldata.IGetKeyspaceRoutingRulesRequest): vtctldata.GetKeyspaceRoutingRulesRequest; + + /** + * Encodes the specified GetKeyspaceRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesRequest.verify|verify} messages. + * @param message GetKeyspaceRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IGetKeyspaceRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified GetKeyspaceRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesRequest.verify|verify} messages. + * @param message GetKeyspaceRoutingRulesRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IGetKeyspaceRoutingRulesRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a GetKeyspaceRoutingRulesRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns GetKeyspaceRoutingRulesRequest + * @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): vtctldata.GetKeyspaceRoutingRulesRequest; + + /** + * Decodes a GetKeyspaceRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns GetKeyspaceRoutingRulesRequest + * @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)): vtctldata.GetKeyspaceRoutingRulesRequest; + + /** + * Verifies a GetKeyspaceRoutingRulesRequest 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 GetKeyspaceRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns GetKeyspaceRoutingRulesRequest + */ + public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspaceRoutingRulesRequest; + + /** + * Creates a plain object from a GetKeyspaceRoutingRulesRequest message. Also converts values to other types if specified. + * @param message GetKeyspaceRoutingRulesRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.GetKeyspaceRoutingRulesRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this GetKeyspaceRoutingRulesRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for GetKeyspaceRoutingRulesRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a GetKeyspaceRoutingRulesResponse. */ + interface IGetKeyspaceRoutingRulesResponse { + + /** GetKeyspaceRoutingRulesResponse keyspace_routing_rules */ + keyspace_routing_rules?: (vschema.IKeyspaceRoutingRules|null); + } + + /** Represents a GetKeyspaceRoutingRulesResponse. */ + class GetKeyspaceRoutingRulesResponse implements IGetKeyspaceRoutingRulesResponse { + + /** + * Constructs a new GetKeyspaceRoutingRulesResponse. + * @param [properties] Properties to set + */ + constructor(properties?: vtctldata.IGetKeyspaceRoutingRulesResponse); + + /** GetKeyspaceRoutingRulesResponse keyspace_routing_rules. */ + public keyspace_routing_rules?: (vschema.IKeyspaceRoutingRules|null); + + /** + * Creates a new GetKeyspaceRoutingRulesResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns GetKeyspaceRoutingRulesResponse instance + */ + public static create(properties?: vtctldata.IGetKeyspaceRoutingRulesResponse): vtctldata.GetKeyspaceRoutingRulesResponse; + + /** + * Encodes the specified GetKeyspaceRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesResponse.verify|verify} messages. + * @param message GetKeyspaceRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: vtctldata.IGetKeyspaceRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified GetKeyspaceRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesResponse.verify|verify} messages. + * @param message GetKeyspaceRoutingRulesResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: vtctldata.IGetKeyspaceRoutingRulesResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a GetKeyspaceRoutingRulesResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns GetKeyspaceRoutingRulesResponse + * @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): vtctldata.GetKeyspaceRoutingRulesResponse; + + /** + * Decodes a GetKeyspaceRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns GetKeyspaceRoutingRulesResponse + * @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)): vtctldata.GetKeyspaceRoutingRulesResponse; + + /** + * Verifies a GetKeyspaceRoutingRulesResponse 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 GetKeyspaceRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns GetKeyspaceRoutingRulesResponse + */ + public static fromObject(object: { [k: string]: any }): vtctldata.GetKeyspaceRoutingRulesResponse; + + /** + * Creates a plain object from a GetKeyspaceRoutingRulesResponse message. Also converts values to other types if specified. + * @param message GetKeyspaceRoutingRulesResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: vtctldata.GetKeyspaceRoutingRulesResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this GetKeyspaceRoutingRulesResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for GetKeyspaceRoutingRulesResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of a GetRoutingRulesRequest. */ interface IGetRoutingRulesRequest { } @@ -59220,6 +60050,9 @@ export namespace vtctldata { /** MoveTablesCreateRequest atomic_copy */ atomic_copy?: (boolean|null); + + /** MoveTablesCreateRequest workflow_options */ + workflow_options?: (vtctldata.IWorkflowOptions|null); } /** Represents a MoveTablesCreateRequest. */ @@ -59288,6 +60121,9 @@ export namespace vtctldata { /** MoveTablesCreateRequest atomic_copy. */ public atomic_copy: boolean; + /** MoveTablesCreateRequest workflow_options. */ + public workflow_options?: (vtctldata.IWorkflowOptions|null); + /** * 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 b8b9d910091..18bad5727a3 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -62757,6 +62757,7 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { * @property {boolean|null} [defer_secondary_keys] CreateVReplicationWorkflowRequest defer_secondary_keys * @property {boolean|null} [auto_start] CreateVReplicationWorkflowRequest auto_start * @property {boolean|null} [stop_after_copy] CreateVReplicationWorkflowRequest stop_after_copy + * @property {string|null} [options] CreateVReplicationWorkflowRequest options */ /** @@ -62857,6 +62858,14 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { */ CreateVReplicationWorkflowRequest.prototype.stop_after_copy = false; + /** + * CreateVReplicationWorkflowRequest options. + * @member {string} options + * @memberof tabletmanagerdata.CreateVReplicationWorkflowRequest + * @instance + */ + CreateVReplicationWorkflowRequest.prototype.options = ""; + /** * Creates a new CreateVReplicationWorkflowRequest instance using the specified properties. * @function create @@ -62907,6 +62916,8 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { writer.uint32(/* id 9, wireType 0 =*/72).bool(message.auto_start); if (message.stop_after_copy != null && Object.hasOwnProperty.call(message, "stop_after_copy")) writer.uint32(/* id 10, wireType 0 =*/80).bool(message.stop_after_copy); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + writer.uint32(/* id 11, wireType 2 =*/90).string(message.options); return writer; }; @@ -62992,6 +63003,10 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.stop_after_copy = reader.bool(); break; } + case 11: { + message.options = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -63106,6 +63121,9 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) if (typeof message.stop_after_copy !== "boolean") return "stop_after_copy: boolean expected"; + if (message.options != null && message.hasOwnProperty("options")) + if (!$util.isString(message.options)) + return "options: string expected"; return null; }; @@ -63275,6 +63293,8 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.auto_start = Boolean(object.auto_start); if (object.stop_after_copy != null) message.stop_after_copy = Boolean(object.stop_after_copy); + if (object.options != null) + message.options = String(object.options); return message; }; @@ -63304,6 +63324,7 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { object.defer_secondary_keys = false; object.auto_start = false; object.stop_after_copy = false; + object.options = ""; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; @@ -63334,6 +63355,8 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { object.auto_start = message.auto_start; if (message.stop_after_copy != null && message.hasOwnProperty("stop_after_copy")) object.stop_after_copy = message.stop_after_copy; + if (message.options != null && message.hasOwnProperty("options")) + object.options = message.options; return object; }; @@ -65318,6 +65341,7 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { * @property {binlogdata.VReplicationWorkflowSubType|null} [workflow_sub_type] ReadVReplicationWorkflowResponse workflow_sub_type * @property {boolean|null} [defer_secondary_keys] ReadVReplicationWorkflowResponse defer_secondary_keys * @property {Array.|null} [streams] ReadVReplicationWorkflowResponse streams + * @property {string|null} [options] ReadVReplicationWorkflowResponse options */ /** @@ -65417,6 +65441,14 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { */ ReadVReplicationWorkflowResponse.prototype.streams = $util.emptyArray; + /** + * ReadVReplicationWorkflowResponse options. + * @member {string} options + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.options = ""; + /** * Creates a new ReadVReplicationWorkflowResponse instance using the specified properties. * @function create @@ -65466,6 +65498,8 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { if (message.streams != null && message.streams.length) for (let i = 0; i < message.streams.length; ++i) $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.encode(message.streams[i], writer.uint32(/* id 11, wireType 2 =*/90).fork()).ldelim(); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + writer.uint32(/* id 12, wireType 2 =*/98).string(message.options); return writer; }; @@ -65549,6 +65583,10 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.streams.push($root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.decode(reader, reader.uint32())); break; } + case 12: { + message.options = reader.string(); + break; + } default: reader.skipType(tag & 7); break; @@ -65659,6 +65697,9 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { return "streams." + error; } } + if (message.options != null && message.hasOwnProperty("options")) + if (!$util.isString(message.options)) + return "options: string expected"; return null; }; @@ -65823,6 +65864,8 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.streams[i] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.fromObject(object.streams[i]); } } + if (object.options != null) + message.options = String(object.options); return message; }; @@ -65852,6 +65895,7 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { object.workflow_type = options.enums === String ? "Materialize" : 0; object.workflow_sub_type = options.enums === String ? "None" : 0; object.defer_secondary_keys = false; + object.options = ""; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; @@ -65879,6 +65923,8 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { for (let j = 0; j < message.streams.length; ++j) object.streams[j] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.toObject(message.streams[j], options); } + if (message.options != null && message.hasOwnProperty("options")) + object.options = message.options; return object; }; @@ -103986,6 +104032,7 @@ export const vschema = $root.vschema = (() => { * @property {Object.|null} [tables] Keyspace tables * @property {boolean|null} [require_explicit_routing] Keyspace require_explicit_routing * @property {vschema.Keyspace.ForeignKeyMode|null} [foreign_key_mode] Keyspace foreign_key_mode + * @property {vschema.IMultiTenantSpec|null} [multi_tenant_spec] Keyspace multi_tenant_spec */ /** @@ -104045,6 +104092,14 @@ export const vschema = $root.vschema = (() => { */ Keyspace.prototype.foreign_key_mode = 0; + /** + * Keyspace multi_tenant_spec. + * @member {vschema.IMultiTenantSpec|null|undefined} multi_tenant_spec + * @memberof vschema.Keyspace + * @instance + */ + Keyspace.prototype.multi_tenant_spec = null; + /** * Creates a new Keyspace instance using the specified properties. * @function create @@ -104085,6 +104140,8 @@ export const vschema = $root.vschema = (() => { writer.uint32(/* id 4, wireType 0 =*/32).bool(message.require_explicit_routing); if (message.foreign_key_mode != null && Object.hasOwnProperty.call(message, "foreign_key_mode")) writer.uint32(/* id 5, wireType 0 =*/40).int32(message.foreign_key_mode); + if (message.multi_tenant_spec != null && Object.hasOwnProperty.call(message, "multi_tenant_spec")) + $root.vschema.MultiTenantSpec.encode(message.multi_tenant_spec, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); return writer; }; @@ -104177,6 +104234,10 @@ export const vschema = $root.vschema = (() => { message.foreign_key_mode = reader.int32(); break; } + case 6: { + message.multi_tenant_spec = $root.vschema.MultiTenantSpec.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -104248,6 +104309,11 @@ export const vschema = $root.vschema = (() => { case 3: break; } + if (message.multi_tenant_spec != null && message.hasOwnProperty("multi_tenant_spec")) { + let error = $root.vschema.MultiTenantSpec.verify(message.multi_tenant_spec); + if (error) + return "multi_tenant_spec." + error; + } return null; }; @@ -104311,6 +104377,11 @@ export const vschema = $root.vschema = (() => { message.foreign_key_mode = 3; break; } + if (object.multi_tenant_spec != null) { + if (typeof object.multi_tenant_spec !== "object") + throw TypeError(".vschema.Keyspace.multi_tenant_spec: object expected"); + message.multi_tenant_spec = $root.vschema.MultiTenantSpec.fromObject(object.multi_tenant_spec); + } return message; }; @@ -104335,6 +104406,7 @@ export const vschema = $root.vschema = (() => { object.sharded = false; object.require_explicit_routing = false; object.foreign_key_mode = options.enums === String ? "unspecified" : 0; + object.multi_tenant_spec = null; } if (message.sharded != null && message.hasOwnProperty("sharded")) object.sharded = message.sharded; @@ -104353,6 +104425,8 @@ export const vschema = $root.vschema = (() => { object.require_explicit_routing = message.require_explicit_routing; if (message.foreign_key_mode != null && message.hasOwnProperty("foreign_key_mode")) object.foreign_key_mode = options.enums === String ? $root.vschema.Keyspace.ForeignKeyMode[message.foreign_key_mode] === undefined ? message.foreign_key_mode : $root.vschema.Keyspace.ForeignKeyMode[message.foreign_key_mode] : message.foreign_key_mode; + if (message.multi_tenant_spec != null && message.hasOwnProperty("multi_tenant_spec")) + object.multi_tenant_spec = $root.vschema.MultiTenantSpec.toObject(message.multi_tenant_spec, options); return object; }; @@ -104403,6 +104477,417 @@ export const vschema = $root.vschema = (() => { return Keyspace; })(); + vschema.MultiTenantSpec = (function() { + + /** + * Properties of a MultiTenantSpec. + * @memberof vschema + * @interface IMultiTenantSpec + * @property {string|null} [tenant_id_column_name] MultiTenantSpec tenant_id_column_name + * @property {query.Type|null} [tenant_id_column_type] MultiTenantSpec tenant_id_column_type + */ + + /** + * Constructs a new MultiTenantSpec. + * @memberof vschema + * @classdesc Represents a MultiTenantSpec. + * @implements IMultiTenantSpec + * @constructor + * @param {vschema.IMultiTenantSpec=} [properties] Properties to set + */ + function MultiTenantSpec(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]]; + } + + /** + * MultiTenantSpec tenant_id_column_name. + * @member {string} tenant_id_column_name + * @memberof vschema.MultiTenantSpec + * @instance + */ + MultiTenantSpec.prototype.tenant_id_column_name = ""; + + /** + * MultiTenantSpec tenant_id_column_type. + * @member {query.Type} tenant_id_column_type + * @memberof vschema.MultiTenantSpec + * @instance + */ + MultiTenantSpec.prototype.tenant_id_column_type = 0; + + /** + * Creates a new MultiTenantSpec instance using the specified properties. + * @function create + * @memberof vschema.MultiTenantSpec + * @static + * @param {vschema.IMultiTenantSpec=} [properties] Properties to set + * @returns {vschema.MultiTenantSpec} MultiTenantSpec instance + */ + MultiTenantSpec.create = function create(properties) { + return new MultiTenantSpec(properties); + }; + + /** + * Encodes the specified MultiTenantSpec message. Does not implicitly {@link vschema.MultiTenantSpec.verify|verify} messages. + * @function encode + * @memberof vschema.MultiTenantSpec + * @static + * @param {vschema.IMultiTenantSpec} message MultiTenantSpec message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + MultiTenantSpec.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.tenant_id_column_name != null && Object.hasOwnProperty.call(message, "tenant_id_column_name")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tenant_id_column_name); + if (message.tenant_id_column_type != null && Object.hasOwnProperty.call(message, "tenant_id_column_type")) + writer.uint32(/* id 2, wireType 0 =*/16).int32(message.tenant_id_column_type); + return writer; + }; + + /** + * Encodes the specified MultiTenantSpec message, length delimited. Does not implicitly {@link vschema.MultiTenantSpec.verify|verify} messages. + * @function encodeDelimited + * @memberof vschema.MultiTenantSpec + * @static + * @param {vschema.IMultiTenantSpec} message MultiTenantSpec message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + MultiTenantSpec.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a MultiTenantSpec message from the specified reader or buffer. + * @function decode + * @memberof vschema.MultiTenantSpec + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vschema.MultiTenantSpec} MultiTenantSpec + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + MultiTenantSpec.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.vschema.MultiTenantSpec(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.tenant_id_column_name = reader.string(); + break; + } + case 2: { + message.tenant_id_column_type = reader.int32(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a MultiTenantSpec message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vschema.MultiTenantSpec + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vschema.MultiTenantSpec} MultiTenantSpec + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + MultiTenantSpec.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a MultiTenantSpec message. + * @function verify + * @memberof vschema.MultiTenantSpec + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + MultiTenantSpec.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.tenant_id_column_name != null && message.hasOwnProperty("tenant_id_column_name")) + if (!$util.isString(message.tenant_id_column_name)) + return "tenant_id_column_name: string expected"; + if (message.tenant_id_column_type != null && message.hasOwnProperty("tenant_id_column_type")) + switch (message.tenant_id_column_type) { + default: + return "tenant_id_column_type: enum value expected"; + case 0: + case 257: + case 770: + case 259: + case 772: + case 261: + case 774: + case 263: + case 776: + case 265: + case 778: + case 1035: + case 1036: + case 2061: + case 2062: + case 2063: + case 2064: + case 785: + case 18: + case 6163: + case 10260: + case 6165: + case 10262: + case 6167: + case 10264: + case 2073: + case 2074: + case 2075: + case 28: + case 2077: + case 2078: + case 31: + case 4128: + case 4129: + case 4130: + break; + } + return null; + }; + + /** + * Creates a MultiTenantSpec message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vschema.MultiTenantSpec + * @static + * @param {Object.} object Plain object + * @returns {vschema.MultiTenantSpec} MultiTenantSpec + */ + MultiTenantSpec.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.MultiTenantSpec) + return object; + let message = new $root.vschema.MultiTenantSpec(); + if (object.tenant_id_column_name != null) + message.tenant_id_column_name = String(object.tenant_id_column_name); + switch (object.tenant_id_column_type) { + default: + if (typeof object.tenant_id_column_type === "number") { + message.tenant_id_column_type = object.tenant_id_column_type; + break; + } + break; + case "NULL_TYPE": + case 0: + message.tenant_id_column_type = 0; + break; + case "INT8": + case 257: + message.tenant_id_column_type = 257; + break; + case "UINT8": + case 770: + message.tenant_id_column_type = 770; + break; + case "INT16": + case 259: + message.tenant_id_column_type = 259; + break; + case "UINT16": + case 772: + message.tenant_id_column_type = 772; + break; + case "INT24": + case 261: + message.tenant_id_column_type = 261; + break; + case "UINT24": + case 774: + message.tenant_id_column_type = 774; + break; + case "INT32": + case 263: + message.tenant_id_column_type = 263; + break; + case "UINT32": + case 776: + message.tenant_id_column_type = 776; + break; + case "INT64": + case 265: + message.tenant_id_column_type = 265; + break; + case "UINT64": + case 778: + message.tenant_id_column_type = 778; + break; + case "FLOAT32": + case 1035: + message.tenant_id_column_type = 1035; + break; + case "FLOAT64": + case 1036: + message.tenant_id_column_type = 1036; + break; + case "TIMESTAMP": + case 2061: + message.tenant_id_column_type = 2061; + break; + case "DATE": + case 2062: + message.tenant_id_column_type = 2062; + break; + case "TIME": + case 2063: + message.tenant_id_column_type = 2063; + break; + case "DATETIME": + case 2064: + message.tenant_id_column_type = 2064; + break; + case "YEAR": + case 785: + message.tenant_id_column_type = 785; + break; + case "DECIMAL": + case 18: + message.tenant_id_column_type = 18; + break; + case "TEXT": + case 6163: + message.tenant_id_column_type = 6163; + break; + case "BLOB": + case 10260: + message.tenant_id_column_type = 10260; + break; + case "VARCHAR": + case 6165: + message.tenant_id_column_type = 6165; + break; + case "VARBINARY": + case 10262: + message.tenant_id_column_type = 10262; + break; + case "CHAR": + case 6167: + message.tenant_id_column_type = 6167; + break; + case "BINARY": + case 10264: + message.tenant_id_column_type = 10264; + break; + case "BIT": + case 2073: + message.tenant_id_column_type = 2073; + break; + case "ENUM": + case 2074: + message.tenant_id_column_type = 2074; + break; + case "SET": + case 2075: + message.tenant_id_column_type = 2075; + break; + case "TUPLE": + case 28: + message.tenant_id_column_type = 28; + break; + case "GEOMETRY": + case 2077: + message.tenant_id_column_type = 2077; + break; + case "JSON": + case 2078: + message.tenant_id_column_type = 2078; + break; + case "EXPRESSION": + case 31: + message.tenant_id_column_type = 31; + break; + case "HEXNUM": + case 4128: + message.tenant_id_column_type = 4128; + break; + case "HEXVAL": + case 4129: + message.tenant_id_column_type = 4129; + break; + case "BITNUM": + case 4130: + message.tenant_id_column_type = 4130; + break; + } + return message; + }; + + /** + * Creates a plain object from a MultiTenantSpec message. Also converts values to other types if specified. + * @function toObject + * @memberof vschema.MultiTenantSpec + * @static + * @param {vschema.MultiTenantSpec} message MultiTenantSpec + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + MultiTenantSpec.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tenant_id_column_name = ""; + object.tenant_id_column_type = options.enums === String ? "NULL_TYPE" : 0; + } + if (message.tenant_id_column_name != null && message.hasOwnProperty("tenant_id_column_name")) + object.tenant_id_column_name = message.tenant_id_column_name; + if (message.tenant_id_column_type != null && message.hasOwnProperty("tenant_id_column_type")) + object.tenant_id_column_type = options.enums === String ? $root.query.Type[message.tenant_id_column_type] === undefined ? message.tenant_id_column_type : $root.query.Type[message.tenant_id_column_type] : message.tenant_id_column_type; + return object; + }; + + /** + * Converts this MultiTenantSpec to JSON. + * @function toJSON + * @memberof vschema.MultiTenantSpec + * @instance + * @returns {Object.} JSON object + */ + MultiTenantSpec.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for MultiTenantSpec + * @function getTypeUrl + * @memberof vschema.MultiTenantSpec + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + MultiTenantSpec.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vschema.MultiTenantSpec"; + }; + + return MultiTenantSpec; + })(); + vschema.Vindex = (function() { /** @@ -106191,6 +106676,7 @@ export const vschema = $root.vschema = (() => { * @property {Object.|null} [keyspaces] SrvVSchema keyspaces * @property {vschema.IRoutingRules|null} [routing_rules] SrvVSchema routing_rules * @property {vschema.IShardRoutingRules|null} [shard_routing_rules] SrvVSchema shard_routing_rules + * @property {vschema.IKeyspaceRoutingRules|null} [keyspace_routing_rules] SrvVSchema keyspace_routing_rules */ /** @@ -106233,6 +106719,14 @@ export const vschema = $root.vschema = (() => { */ SrvVSchema.prototype.shard_routing_rules = null; + /** + * SrvVSchema keyspace_routing_rules. + * @member {vschema.IKeyspaceRoutingRules|null|undefined} keyspace_routing_rules + * @memberof vschema.SrvVSchema + * @instance + */ + SrvVSchema.prototype.keyspace_routing_rules = null; + /** * Creates a new SrvVSchema instance using the specified properties. * @function create @@ -106266,6 +106760,8 @@ export const vschema = $root.vschema = (() => { $root.vschema.RoutingRules.encode(message.routing_rules, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); if (message.shard_routing_rules != null && Object.hasOwnProperty.call(message, "shard_routing_rules")) $root.vschema.ShardRoutingRules.encode(message.shard_routing_rules, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.keyspace_routing_rules != null && Object.hasOwnProperty.call(message, "keyspace_routing_rules")) + $root.vschema.KeyspaceRoutingRules.encode(message.keyspace_routing_rules, writer.uint32(/* id 4, wireType 2 =*/34).fork()).ldelim(); return writer; }; @@ -106331,6 +106827,10 @@ export const vschema = $root.vschema = (() => { message.shard_routing_rules = $root.vschema.ShardRoutingRules.decode(reader, reader.uint32()); break; } + case 4: { + message.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -106386,6 +106886,11 @@ export const vschema = $root.vschema = (() => { if (error) return "shard_routing_rules." + error; } + if (message.keyspace_routing_rules != null && message.hasOwnProperty("keyspace_routing_rules")) { + let error = $root.vschema.KeyspaceRoutingRules.verify(message.keyspace_routing_rules); + if (error) + return "keyspace_routing_rules." + error; + } return null; }; @@ -106421,6 +106926,11 @@ export const vschema = $root.vschema = (() => { throw TypeError(".vschema.SrvVSchema.shard_routing_rules: object expected"); message.shard_routing_rules = $root.vschema.ShardRoutingRules.fromObject(object.shard_routing_rules); } + if (object.keyspace_routing_rules != null) { + if (typeof object.keyspace_routing_rules !== "object") + throw TypeError(".vschema.SrvVSchema.keyspace_routing_rules: object expected"); + message.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.fromObject(object.keyspace_routing_rules); + } return message; }; @@ -106442,6 +106952,7 @@ export const vschema = $root.vschema = (() => { if (options.defaults) { object.routing_rules = null; object.shard_routing_rules = null; + object.keyspace_routing_rules = null; } let keys2; if (message.keyspaces && (keys2 = Object.keys(message.keyspaces)).length) { @@ -106453,6 +106964,8 @@ export const vschema = $root.vschema = (() => { object.routing_rules = $root.vschema.RoutingRules.toObject(message.routing_rules, options); if (message.shard_routing_rules != null && message.hasOwnProperty("shard_routing_rules")) object.shard_routing_rules = $root.vschema.ShardRoutingRules.toObject(message.shard_routing_rules, options); + if (message.keyspace_routing_rules != null && message.hasOwnProperty("keyspace_routing_rules")) + object.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.toObject(message.keyspace_routing_rules, options); return object; }; @@ -106959,6 +107472,457 @@ export const vschema = $root.vschema = (() => { return ShardRoutingRule; })(); + vschema.KeyspaceRoutingRules = (function() { + + /** + * Properties of a KeyspaceRoutingRules. + * @memberof vschema + * @interface IKeyspaceRoutingRules + * @property {Array.|null} [rules] KeyspaceRoutingRules rules + */ + + /** + * Constructs a new KeyspaceRoutingRules. + * @memberof vschema + * @classdesc Represents a KeyspaceRoutingRules. + * @implements IKeyspaceRoutingRules + * @constructor + * @param {vschema.IKeyspaceRoutingRules=} [properties] Properties to set + */ + function KeyspaceRoutingRules(properties) { + this.rules = []; + 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]]; + } + + /** + * KeyspaceRoutingRules rules. + * @member {Array.} rules + * @memberof vschema.KeyspaceRoutingRules + * @instance + */ + KeyspaceRoutingRules.prototype.rules = $util.emptyArray; + + /** + * Creates a new KeyspaceRoutingRules instance using the specified properties. + * @function create + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {vschema.IKeyspaceRoutingRules=} [properties] Properties to set + * @returns {vschema.KeyspaceRoutingRules} KeyspaceRoutingRules instance + */ + KeyspaceRoutingRules.create = function create(properties) { + return new KeyspaceRoutingRules(properties); + }; + + /** + * Encodes the specified KeyspaceRoutingRules message. Does not implicitly {@link vschema.KeyspaceRoutingRules.verify|verify} messages. + * @function encode + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {vschema.IKeyspaceRoutingRules} message KeyspaceRoutingRules message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + KeyspaceRoutingRules.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.rules != null && message.rules.length) + for (let i = 0; i < message.rules.length; ++i) + $root.vschema.KeyspaceRoutingRule.encode(message.rules[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified KeyspaceRoutingRules message, length delimited. Does not implicitly {@link vschema.KeyspaceRoutingRules.verify|verify} messages. + * @function encodeDelimited + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {vschema.IKeyspaceRoutingRules} message KeyspaceRoutingRules message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + KeyspaceRoutingRules.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a KeyspaceRoutingRules message from the specified reader or buffer. + * @function decode + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vschema.KeyspaceRoutingRules} KeyspaceRoutingRules + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + KeyspaceRoutingRules.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.vschema.KeyspaceRoutingRules(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.rules && message.rules.length)) + message.rules = []; + message.rules.push($root.vschema.KeyspaceRoutingRule.decode(reader, reader.uint32())); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a KeyspaceRoutingRules message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vschema.KeyspaceRoutingRules} KeyspaceRoutingRules + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + KeyspaceRoutingRules.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a KeyspaceRoutingRules message. + * @function verify + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + KeyspaceRoutingRules.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.rules != null && message.hasOwnProperty("rules")) { + if (!Array.isArray(message.rules)) + return "rules: array expected"; + for (let i = 0; i < message.rules.length; ++i) { + let error = $root.vschema.KeyspaceRoutingRule.verify(message.rules[i]); + if (error) + return "rules." + error; + } + } + return null; + }; + + /** + * Creates a KeyspaceRoutingRules message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {Object.} object Plain object + * @returns {vschema.KeyspaceRoutingRules} KeyspaceRoutingRules + */ + KeyspaceRoutingRules.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.KeyspaceRoutingRules) + return object; + let message = new $root.vschema.KeyspaceRoutingRules(); + if (object.rules) { + if (!Array.isArray(object.rules)) + throw TypeError(".vschema.KeyspaceRoutingRules.rules: array expected"); + message.rules = []; + for (let i = 0; i < object.rules.length; ++i) { + if (typeof object.rules[i] !== "object") + throw TypeError(".vschema.KeyspaceRoutingRules.rules: object expected"); + message.rules[i] = $root.vschema.KeyspaceRoutingRule.fromObject(object.rules[i]); + } + } + return message; + }; + + /** + * Creates a plain object from a KeyspaceRoutingRules message. Also converts values to other types if specified. + * @function toObject + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {vschema.KeyspaceRoutingRules} message KeyspaceRoutingRules + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + KeyspaceRoutingRules.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.rules = []; + if (message.rules && message.rules.length) { + object.rules = []; + for (let j = 0; j < message.rules.length; ++j) + object.rules[j] = $root.vschema.KeyspaceRoutingRule.toObject(message.rules[j], options); + } + return object; + }; + + /** + * Converts this KeyspaceRoutingRules to JSON. + * @function toJSON + * @memberof vschema.KeyspaceRoutingRules + * @instance + * @returns {Object.} JSON object + */ + KeyspaceRoutingRules.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for KeyspaceRoutingRules + * @function getTypeUrl + * @memberof vschema.KeyspaceRoutingRules + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + KeyspaceRoutingRules.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vschema.KeyspaceRoutingRules"; + }; + + return KeyspaceRoutingRules; + })(); + + vschema.KeyspaceRoutingRule = (function() { + + /** + * Properties of a KeyspaceRoutingRule. + * @memberof vschema + * @interface IKeyspaceRoutingRule + * @property {string|null} [from_keyspace] KeyspaceRoutingRule from_keyspace + * @property {string|null} [to_keyspace] KeyspaceRoutingRule to_keyspace + */ + + /** + * Constructs a new KeyspaceRoutingRule. + * @memberof vschema + * @classdesc Represents a KeyspaceRoutingRule. + * @implements IKeyspaceRoutingRule + * @constructor + * @param {vschema.IKeyspaceRoutingRule=} [properties] Properties to set + */ + function KeyspaceRoutingRule(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]]; + } + + /** + * KeyspaceRoutingRule from_keyspace. + * @member {string} from_keyspace + * @memberof vschema.KeyspaceRoutingRule + * @instance + */ + KeyspaceRoutingRule.prototype.from_keyspace = ""; + + /** + * KeyspaceRoutingRule to_keyspace. + * @member {string} to_keyspace + * @memberof vschema.KeyspaceRoutingRule + * @instance + */ + KeyspaceRoutingRule.prototype.to_keyspace = ""; + + /** + * Creates a new KeyspaceRoutingRule instance using the specified properties. + * @function create + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {vschema.IKeyspaceRoutingRule=} [properties] Properties to set + * @returns {vschema.KeyspaceRoutingRule} KeyspaceRoutingRule instance + */ + KeyspaceRoutingRule.create = function create(properties) { + return new KeyspaceRoutingRule(properties); + }; + + /** + * Encodes the specified KeyspaceRoutingRule message. Does not implicitly {@link vschema.KeyspaceRoutingRule.verify|verify} messages. + * @function encode + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {vschema.IKeyspaceRoutingRule} message KeyspaceRoutingRule message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + KeyspaceRoutingRule.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.from_keyspace != null && Object.hasOwnProperty.call(message, "from_keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.from_keyspace); + if (message.to_keyspace != null && Object.hasOwnProperty.call(message, "to_keyspace")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.to_keyspace); + return writer; + }; + + /** + * Encodes the specified KeyspaceRoutingRule message, length delimited. Does not implicitly {@link vschema.KeyspaceRoutingRule.verify|verify} messages. + * @function encodeDelimited + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {vschema.IKeyspaceRoutingRule} message KeyspaceRoutingRule message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + KeyspaceRoutingRule.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a KeyspaceRoutingRule message from the specified reader or buffer. + * @function decode + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vschema.KeyspaceRoutingRule} KeyspaceRoutingRule + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + KeyspaceRoutingRule.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.vschema.KeyspaceRoutingRule(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.from_keyspace = reader.string(); + break; + } + case 2: { + message.to_keyspace = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a KeyspaceRoutingRule message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vschema.KeyspaceRoutingRule} KeyspaceRoutingRule + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + KeyspaceRoutingRule.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a KeyspaceRoutingRule message. + * @function verify + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + KeyspaceRoutingRule.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.from_keyspace != null && message.hasOwnProperty("from_keyspace")) + if (!$util.isString(message.from_keyspace)) + return "from_keyspace: string expected"; + if (message.to_keyspace != null && message.hasOwnProperty("to_keyspace")) + if (!$util.isString(message.to_keyspace)) + return "to_keyspace: string expected"; + return null; + }; + + /** + * Creates a KeyspaceRoutingRule message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {Object.} object Plain object + * @returns {vschema.KeyspaceRoutingRule} KeyspaceRoutingRule + */ + KeyspaceRoutingRule.fromObject = function fromObject(object) { + if (object instanceof $root.vschema.KeyspaceRoutingRule) + return object; + let message = new $root.vschema.KeyspaceRoutingRule(); + if (object.from_keyspace != null) + message.from_keyspace = String(object.from_keyspace); + if (object.to_keyspace != null) + message.to_keyspace = String(object.to_keyspace); + return message; + }; + + /** + * Creates a plain object from a KeyspaceRoutingRule message. Also converts values to other types if specified. + * @function toObject + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {vschema.KeyspaceRoutingRule} message KeyspaceRoutingRule + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + KeyspaceRoutingRule.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.from_keyspace = ""; + object.to_keyspace = ""; + } + if (message.from_keyspace != null && message.hasOwnProperty("from_keyspace")) + object.from_keyspace = message.from_keyspace; + if (message.to_keyspace != null && message.hasOwnProperty("to_keyspace")) + object.to_keyspace = message.to_keyspace; + return object; + }; + + /** + * Converts this KeyspaceRoutingRule to JSON. + * @function toJSON + * @memberof vschema.KeyspaceRoutingRule + * @instance + * @returns {Object.} JSON object + */ + KeyspaceRoutingRule.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for KeyspaceRoutingRule + * @function getTypeUrl + * @memberof vschema.KeyspaceRoutingRule + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + KeyspaceRoutingRule.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vschema.KeyspaceRoutingRule"; + }; + + return KeyspaceRoutingRule; + })(); + return vschema; })(); @@ -107724,6 +108688,7 @@ export const vtctldata = $root.vtctldata = (() => { * @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 + * @property {vtctldata.IWorkflowOptions|null} [workflow_options] MaterializeSettings workflow_options */ /** @@ -107871,6 +108836,14 @@ export const vtctldata = $root.vtctldata = (() => { */ MaterializeSettings.prototype.atomic_copy = false; + /** + * MaterializeSettings workflow_options. + * @member {vtctldata.IWorkflowOptions|null|undefined} workflow_options + * @memberof vtctldata.MaterializeSettings + * @instance + */ + MaterializeSettings.prototype.workflow_options = null; + /** * Creates a new MaterializeSettings instance using the specified properties. * @function create @@ -107929,6 +108902,8 @@ export const vtctldata = $root.vtctldata = (() => { 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); + if (message.workflow_options != null && Object.hasOwnProperty.call(message, "workflow_options")) + $root.vtctldata.WorkflowOptions.encode(message.workflow_options, writer.uint32(/* id 17, wireType 2 =*/138).fork()).ldelim(); return writer; }; @@ -108031,6 +109006,10 @@ export const vtctldata = $root.vtctldata = (() => { message.atomic_copy = reader.bool(); break; } + case 17: { + message.workflow_options = $root.vtctldata.WorkflowOptions.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -108136,6 +109115,11 @@ export const vtctldata = $root.vtctldata = (() => { if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) if (typeof message.atomic_copy !== "boolean") return "atomic_copy: boolean expected"; + if (message.workflow_options != null && message.hasOwnProperty("workflow_options")) { + let error = $root.vtctldata.WorkflowOptions.verify(message.workflow_options); + if (error) + return "workflow_options." + error; + } return null; }; @@ -108232,6 +109216,11 @@ export const vtctldata = $root.vtctldata = (() => { } if (object.atomic_copy != null) message.atomic_copy = Boolean(object.atomic_copy); + if (object.workflow_options != null) { + if (typeof object.workflow_options !== "object") + throw TypeError(".vtctldata.MaterializeSettings.workflow_options: object expected"); + message.workflow_options = $root.vtctldata.WorkflowOptions.fromObject(object.workflow_options); + } return message; }; @@ -108267,6 +109256,7 @@ export const vtctldata = $root.vtctldata = (() => { object.defer_secondary_keys = false; object.tablet_selection_preference = options.enums === String ? "ANY" : 0; object.atomic_copy = false; + object.workflow_options = null; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; @@ -108306,6 +109296,8 @@ export const vtctldata = $root.vtctldata = (() => { 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; + if (message.workflow_options != null && message.hasOwnProperty("workflow_options")) + object.workflow_options = $root.vtctldata.WorkflowOptions.toObject(message.workflow_options, options); return object; }; @@ -110535,6 +111527,233 @@ export const vtctldata = $root.vtctldata = (() => { return Shard; })(); + vtctldata.WorkflowOptions = (function() { + + /** + * Properties of a WorkflowOptions. + * @memberof vtctldata + * @interface IWorkflowOptions + * @property {string|null} [tenant_id] WorkflowOptions tenant_id + * @property {string|null} [source_keyspace_alias] WorkflowOptions source_keyspace_alias + */ + + /** + * Constructs a new WorkflowOptions. + * @memberof vtctldata + * @classdesc Represents a WorkflowOptions. + * @implements IWorkflowOptions + * @constructor + * @param {vtctldata.IWorkflowOptions=} [properties] Properties to set + */ + function WorkflowOptions(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]]; + } + + /** + * WorkflowOptions tenant_id. + * @member {string} tenant_id + * @memberof vtctldata.WorkflowOptions + * @instance + */ + WorkflowOptions.prototype.tenant_id = ""; + + /** + * WorkflowOptions source_keyspace_alias. + * @member {string} source_keyspace_alias + * @memberof vtctldata.WorkflowOptions + * @instance + */ + WorkflowOptions.prototype.source_keyspace_alias = ""; + + /** + * Creates a new WorkflowOptions instance using the specified properties. + * @function create + * @memberof vtctldata.WorkflowOptions + * @static + * @param {vtctldata.IWorkflowOptions=} [properties] Properties to set + * @returns {vtctldata.WorkflowOptions} WorkflowOptions instance + */ + WorkflowOptions.create = function create(properties) { + return new WorkflowOptions(properties); + }; + + /** + * Encodes the specified WorkflowOptions message. Does not implicitly {@link vtctldata.WorkflowOptions.verify|verify} messages. + * @function encode + * @memberof vtctldata.WorkflowOptions + * @static + * @param {vtctldata.IWorkflowOptions} message WorkflowOptions message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + WorkflowOptions.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.tenant_id != null && Object.hasOwnProperty.call(message, "tenant_id")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tenant_id); + if (message.source_keyspace_alias != null && Object.hasOwnProperty.call(message, "source_keyspace_alias")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_keyspace_alias); + return writer; + }; + + /** + * Encodes the specified WorkflowOptions message, length delimited. Does not implicitly {@link vtctldata.WorkflowOptions.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.WorkflowOptions + * @static + * @param {vtctldata.IWorkflowOptions} message WorkflowOptions message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + WorkflowOptions.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a WorkflowOptions message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.WorkflowOptions + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.WorkflowOptions} WorkflowOptions + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + WorkflowOptions.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.vtctldata.WorkflowOptions(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.tenant_id = reader.string(); + break; + } + case 2: { + message.source_keyspace_alias = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a WorkflowOptions message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.WorkflowOptions + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.WorkflowOptions} WorkflowOptions + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + WorkflowOptions.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a WorkflowOptions message. + * @function verify + * @memberof vtctldata.WorkflowOptions + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + WorkflowOptions.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.tenant_id != null && message.hasOwnProperty("tenant_id")) + if (!$util.isString(message.tenant_id)) + return "tenant_id: string expected"; + if (message.source_keyspace_alias != null && message.hasOwnProperty("source_keyspace_alias")) + if (!$util.isString(message.source_keyspace_alias)) + return "source_keyspace_alias: string expected"; + return null; + }; + + /** + * Creates a WorkflowOptions message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.WorkflowOptions + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.WorkflowOptions} WorkflowOptions + */ + WorkflowOptions.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.WorkflowOptions) + return object; + let message = new $root.vtctldata.WorkflowOptions(); + if (object.tenant_id != null) + message.tenant_id = String(object.tenant_id); + if (object.source_keyspace_alias != null) + message.source_keyspace_alias = String(object.source_keyspace_alias); + return message; + }; + + /** + * Creates a plain object from a WorkflowOptions message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.WorkflowOptions + * @static + * @param {vtctldata.WorkflowOptions} message WorkflowOptions + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + WorkflowOptions.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.tenant_id = ""; + object.source_keyspace_alias = ""; + } + if (message.tenant_id != null && message.hasOwnProperty("tenant_id")) + object.tenant_id = message.tenant_id; + if (message.source_keyspace_alias != null && message.hasOwnProperty("source_keyspace_alias")) + object.source_keyspace_alias = message.source_keyspace_alias; + return object; + }; + + /** + * Converts this WorkflowOptions to JSON. + * @function toJSON + * @memberof vtctldata.WorkflowOptions + * @instance + * @returns {Object.} JSON object + */ + WorkflowOptions.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for WorkflowOptions + * @function getTypeUrl + * @memberof vtctldata.WorkflowOptions + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + WorkflowOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.WorkflowOptions"; + }; + + return WorkflowOptions; + })(); + vtctldata.Workflow = (function() { /** @@ -110550,6 +111769,7 @@ export const vtctldata = $root.vtctldata = (() => { * @property {string|null} [workflow_sub_type] Workflow workflow_sub_type * @property {number|Long|null} [max_v_replication_transaction_lag] Workflow max_v_replication_transaction_lag * @property {boolean|null} [defer_secondary_keys] Workflow defer_secondary_keys + * @property {vtctldata.IWorkflowOptions|null} [options] Workflow options */ /** @@ -110640,6 +111860,14 @@ export const vtctldata = $root.vtctldata = (() => { */ Workflow.prototype.defer_secondary_keys = false; + /** + * Workflow options. + * @member {vtctldata.IWorkflowOptions|null|undefined} options + * @memberof vtctldata.Workflow + * @instance + */ + Workflow.prototype.options = null; + /** * Creates a new Workflow instance using the specified properties. * @function create @@ -110685,6 +111913,8 @@ export const vtctldata = $root.vtctldata = (() => { writer.uint32(/* id 8, wireType 0 =*/64).int64(message.max_v_replication_transaction_lag); if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) writer.uint32(/* id 9, wireType 0 =*/72).bool(message.defer_secondary_keys); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.vtctldata.WorkflowOptions.encode(message.options, writer.uint32(/* id 10, wireType 2 =*/82).fork()).ldelim(); return writer; }; @@ -110774,6 +112004,10 @@ export const vtctldata = $root.vtctldata = (() => { message.defer_secondary_keys = reader.bool(); break; } + case 10: { + message.options = $root.vtctldata.WorkflowOptions.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -110847,6 +112081,11 @@ export const vtctldata = $root.vtctldata = (() => { if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) if (typeof message.defer_secondary_keys !== "boolean") return "defer_secondary_keys: boolean expected"; + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.vtctldata.WorkflowOptions.verify(message.options); + if (error) + return "options." + error; + } return null; }; @@ -110908,6 +112147,11 @@ export const vtctldata = $root.vtctldata = (() => { message.max_v_replication_transaction_lag = new $util.LongBits(object.max_v_replication_transaction_lag.low >>> 0, object.max_v_replication_transaction_lag.high >>> 0).toNumber(); if (object.defer_secondary_keys != null) message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".vtctldata.Workflow.options: object expected"); + message.options = $root.vtctldata.WorkflowOptions.fromObject(object.options); + } return message; }; @@ -110943,6 +112187,7 @@ export const vtctldata = $root.vtctldata = (() => { } else object.max_v_replication_transaction_lag = options.longs === String ? "0" : 0; object.defer_secondary_keys = false; + object.options = null; } if (message.name != null && message.hasOwnProperty("name")) object.name = message.name; @@ -110972,6 +112217,8 @@ export const vtctldata = $root.vtctldata = (() => { object.max_v_replication_transaction_lag = options.longs === String ? $util.Long.prototype.toString.call(message.max_v_replication_transaction_lag) : options.longs === Number ? new $util.LongBits(message.max_v_replication_transaction_lag.low >>> 0, message.max_v_replication_transaction_lag.high >>> 0).toNumber() : message.max_v_replication_transaction_lag; if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) object.defer_secondary_keys = message.defer_secondary_keys; + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.vtctldata.WorkflowOptions.toObject(message.options, options); return object; }; @@ -114160,6 +115407,453 @@ export const vtctldata = $root.vtctldata = (() => { return AddCellsAliasResponse; })(); + vtctldata.ApplyKeyspaceRoutingRulesRequest = (function() { + + /** + * Properties of an ApplyKeyspaceRoutingRulesRequest. + * @memberof vtctldata + * @interface IApplyKeyspaceRoutingRulesRequest + * @property {vschema.IKeyspaceRoutingRules|null} [keyspace_routing_rules] ApplyKeyspaceRoutingRulesRequest keyspace_routing_rules + * @property {boolean|null} [skip_rebuild] ApplyKeyspaceRoutingRulesRequest skip_rebuild + * @property {Array.|null} [rebuild_cells] ApplyKeyspaceRoutingRulesRequest rebuild_cells + */ + + /** + * Constructs a new ApplyKeyspaceRoutingRulesRequest. + * @memberof vtctldata + * @classdesc Represents an ApplyKeyspaceRoutingRulesRequest. + * @implements IApplyKeyspaceRoutingRulesRequest + * @constructor + * @param {vtctldata.IApplyKeyspaceRoutingRulesRequest=} [properties] Properties to set + */ + function ApplyKeyspaceRoutingRulesRequest(properties) { + this.rebuild_cells = []; + 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]]; + } + + /** + * ApplyKeyspaceRoutingRulesRequest keyspace_routing_rules. + * @member {vschema.IKeyspaceRoutingRules|null|undefined} keyspace_routing_rules + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @instance + */ + ApplyKeyspaceRoutingRulesRequest.prototype.keyspace_routing_rules = null; + + /** + * ApplyKeyspaceRoutingRulesRequest skip_rebuild. + * @member {boolean} skip_rebuild + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @instance + */ + ApplyKeyspaceRoutingRulesRequest.prototype.skip_rebuild = false; + + /** + * ApplyKeyspaceRoutingRulesRequest rebuild_cells. + * @member {Array.} rebuild_cells + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @instance + */ + ApplyKeyspaceRoutingRulesRequest.prototype.rebuild_cells = $util.emptyArray; + + /** + * Creates a new ApplyKeyspaceRoutingRulesRequest instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.IApplyKeyspaceRoutingRulesRequest=} [properties] Properties to set + * @returns {vtctldata.ApplyKeyspaceRoutingRulesRequest} ApplyKeyspaceRoutingRulesRequest instance + */ + ApplyKeyspaceRoutingRulesRequest.create = function create(properties) { + return new ApplyKeyspaceRoutingRulesRequest(properties); + }; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesRequest message. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.IApplyKeyspaceRoutingRulesRequest} message ApplyKeyspaceRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyKeyspaceRoutingRulesRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace_routing_rules != null && Object.hasOwnProperty.call(message, "keyspace_routing_rules")) + $root.vschema.KeyspaceRoutingRules.encode(message.keyspace_routing_rules, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.skip_rebuild != null && Object.hasOwnProperty.call(message, "skip_rebuild")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.skip_rebuild); + if (message.rebuild_cells != null && message.rebuild_cells.length) + for (let i = 0; i < message.rebuild_cells.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.rebuild_cells[i]); + return writer; + }; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.IApplyKeyspaceRoutingRulesRequest} message ApplyKeyspaceRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyKeyspaceRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplyKeyspaceRoutingRulesRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyKeyspaceRoutingRulesRequest} ApplyKeyspaceRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyKeyspaceRoutingRulesRequest.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.vtctldata.ApplyKeyspaceRoutingRulesRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.decode(reader, reader.uint32()); + break; + } + case 2: { + message.skip_rebuild = reader.bool(); + break; + } + case 3: { + if (!(message.rebuild_cells && message.rebuild_cells.length)) + message.rebuild_cells = []; + message.rebuild_cells.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an ApplyKeyspaceRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyKeyspaceRoutingRulesRequest} ApplyKeyspaceRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyKeyspaceRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplyKeyspaceRoutingRulesRequest message. + * @function verify + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyKeyspaceRoutingRulesRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace_routing_rules != null && message.hasOwnProperty("keyspace_routing_rules")) { + let error = $root.vschema.KeyspaceRoutingRules.verify(message.keyspace_routing_rules); + if (error) + return "keyspace_routing_rules." + error; + } + if (message.skip_rebuild != null && message.hasOwnProperty("skip_rebuild")) + if (typeof message.skip_rebuild !== "boolean") + return "skip_rebuild: boolean expected"; + if (message.rebuild_cells != null && message.hasOwnProperty("rebuild_cells")) { + if (!Array.isArray(message.rebuild_cells)) + return "rebuild_cells: array expected"; + for (let i = 0; i < message.rebuild_cells.length; ++i) + if (!$util.isString(message.rebuild_cells[i])) + return "rebuild_cells: string[] expected"; + } + return null; + }; + + /** + * Creates an ApplyKeyspaceRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyKeyspaceRoutingRulesRequest} ApplyKeyspaceRoutingRulesRequest + */ + ApplyKeyspaceRoutingRulesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyKeyspaceRoutingRulesRequest) + return object; + let message = new $root.vtctldata.ApplyKeyspaceRoutingRulesRequest(); + if (object.keyspace_routing_rules != null) { + if (typeof object.keyspace_routing_rules !== "object") + throw TypeError(".vtctldata.ApplyKeyspaceRoutingRulesRequest.keyspace_routing_rules: object expected"); + message.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.fromObject(object.keyspace_routing_rules); + } + if (object.skip_rebuild != null) + message.skip_rebuild = Boolean(object.skip_rebuild); + if (object.rebuild_cells) { + if (!Array.isArray(object.rebuild_cells)) + throw TypeError(".vtctldata.ApplyKeyspaceRoutingRulesRequest.rebuild_cells: array expected"); + message.rebuild_cells = []; + for (let i = 0; i < object.rebuild_cells.length; ++i) + message.rebuild_cells[i] = String(object.rebuild_cells[i]); + } + return message; + }; + + /** + * Creates a plain object from an ApplyKeyspaceRoutingRulesRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.ApplyKeyspaceRoutingRulesRequest} message ApplyKeyspaceRoutingRulesRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyKeyspaceRoutingRulesRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.rebuild_cells = []; + if (options.defaults) { + object.keyspace_routing_rules = null; + object.skip_rebuild = false; + } + if (message.keyspace_routing_rules != null && message.hasOwnProperty("keyspace_routing_rules")) + object.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.toObject(message.keyspace_routing_rules, options); + if (message.skip_rebuild != null && message.hasOwnProperty("skip_rebuild")) + object.skip_rebuild = message.skip_rebuild; + if (message.rebuild_cells && message.rebuild_cells.length) { + object.rebuild_cells = []; + for (let j = 0; j < message.rebuild_cells.length; ++j) + object.rebuild_cells[j] = message.rebuild_cells[j]; + } + return object; + }; + + /** + * Converts this ApplyKeyspaceRoutingRulesRequest to JSON. + * @function toJSON + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @instance + * @returns {Object.} JSON object + */ + ApplyKeyspaceRoutingRulesRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplyKeyspaceRoutingRulesRequest + * @function getTypeUrl + * @memberof vtctldata.ApplyKeyspaceRoutingRulesRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyKeyspaceRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyKeyspaceRoutingRulesRequest"; + }; + + return ApplyKeyspaceRoutingRulesRequest; + })(); + + vtctldata.ApplyKeyspaceRoutingRulesResponse = (function() { + + /** + * Properties of an ApplyKeyspaceRoutingRulesResponse. + * @memberof vtctldata + * @interface IApplyKeyspaceRoutingRulesResponse + */ + + /** + * Constructs a new ApplyKeyspaceRoutingRulesResponse. + * @memberof vtctldata + * @classdesc Represents an ApplyKeyspaceRoutingRulesResponse. + * @implements IApplyKeyspaceRoutingRulesResponse + * @constructor + * @param {vtctldata.IApplyKeyspaceRoutingRulesResponse=} [properties] Properties to set + */ + function ApplyKeyspaceRoutingRulesResponse(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]]; + } + + /** + * Creates a new ApplyKeyspaceRoutingRulesResponse instance using the specified properties. + * @function create + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.IApplyKeyspaceRoutingRulesResponse=} [properties] Properties to set + * @returns {vtctldata.ApplyKeyspaceRoutingRulesResponse} ApplyKeyspaceRoutingRulesResponse instance + */ + ApplyKeyspaceRoutingRulesResponse.create = function create(properties) { + return new ApplyKeyspaceRoutingRulesResponse(properties); + }; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesResponse message. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.IApplyKeyspaceRoutingRulesResponse} message ApplyKeyspaceRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyKeyspaceRoutingRulesResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; + + /** + * Encodes the specified ApplyKeyspaceRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.ApplyKeyspaceRoutingRulesResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.IApplyKeyspaceRoutingRulesResponse} message ApplyKeyspaceRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ApplyKeyspaceRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes an ApplyKeyspaceRoutingRulesResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.ApplyKeyspaceRoutingRulesResponse} ApplyKeyspaceRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyKeyspaceRoutingRulesResponse.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.vtctldata.ApplyKeyspaceRoutingRulesResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an ApplyKeyspaceRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.ApplyKeyspaceRoutingRulesResponse} ApplyKeyspaceRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ApplyKeyspaceRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an ApplyKeyspaceRoutingRulesResponse message. + * @function verify + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ApplyKeyspaceRoutingRulesResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates an ApplyKeyspaceRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.ApplyKeyspaceRoutingRulesResponse} ApplyKeyspaceRoutingRulesResponse + */ + ApplyKeyspaceRoutingRulesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.ApplyKeyspaceRoutingRulesResponse) + return object; + return new $root.vtctldata.ApplyKeyspaceRoutingRulesResponse(); + }; + + /** + * Creates a plain object from an ApplyKeyspaceRoutingRulesResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.ApplyKeyspaceRoutingRulesResponse} message ApplyKeyspaceRoutingRulesResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ApplyKeyspaceRoutingRulesResponse.toObject = function toObject() { + return {}; + }; + + /** + * Converts this ApplyKeyspaceRoutingRulesResponse to JSON. + * @function toJSON + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @instance + * @returns {Object.} JSON object + */ + ApplyKeyspaceRoutingRulesResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ApplyKeyspaceRoutingRulesResponse + * @function getTypeUrl + * @memberof vtctldata.ApplyKeyspaceRoutingRulesResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ApplyKeyspaceRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.ApplyKeyspaceRoutingRulesResponse"; + }; + + return ApplyKeyspaceRoutingRulesResponse; + })(); + vtctldata.ApplyRoutingRulesRequest = (function() { /** @@ -130126,6 +131820,389 @@ export const vtctldata = $root.vtctldata = (() => { return GetPermissionsResponse; })(); + vtctldata.GetKeyspaceRoutingRulesRequest = (function() { + + /** + * Properties of a GetKeyspaceRoutingRulesRequest. + * @memberof vtctldata + * @interface IGetKeyspaceRoutingRulesRequest + */ + + /** + * Constructs a new GetKeyspaceRoutingRulesRequest. + * @memberof vtctldata + * @classdesc Represents a GetKeyspaceRoutingRulesRequest. + * @implements IGetKeyspaceRoutingRulesRequest + * @constructor + * @param {vtctldata.IGetKeyspaceRoutingRulesRequest=} [properties] Properties to set + */ + function GetKeyspaceRoutingRulesRequest(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]]; + } + + /** + * Creates a new GetKeyspaceRoutingRulesRequest instance using the specified properties. + * @function create + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.IGetKeyspaceRoutingRulesRequest=} [properties] Properties to set + * @returns {vtctldata.GetKeyspaceRoutingRulesRequest} GetKeyspaceRoutingRulesRequest instance + */ + GetKeyspaceRoutingRulesRequest.create = function create(properties) { + return new GetKeyspaceRoutingRulesRequest(properties); + }; + + /** + * Encodes the specified GetKeyspaceRoutingRulesRequest message. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesRequest.verify|verify} messages. + * @function encode + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.IGetKeyspaceRoutingRulesRequest} message GetKeyspaceRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + GetKeyspaceRoutingRulesRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + return writer; + }; + + /** + * Encodes the specified GetKeyspaceRoutingRulesRequest message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.IGetKeyspaceRoutingRulesRequest} message GetKeyspaceRoutingRulesRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + GetKeyspaceRoutingRulesRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a GetKeyspaceRoutingRulesRequest message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.GetKeyspaceRoutingRulesRequest} GetKeyspaceRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + GetKeyspaceRoutingRulesRequest.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.vtctldata.GetKeyspaceRoutingRulesRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a GetKeyspaceRoutingRulesRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.GetKeyspaceRoutingRulesRequest} GetKeyspaceRoutingRulesRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + GetKeyspaceRoutingRulesRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a GetKeyspaceRoutingRulesRequest message. + * @function verify + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + GetKeyspaceRoutingRulesRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + return null; + }; + + /** + * Creates a GetKeyspaceRoutingRulesRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.GetKeyspaceRoutingRulesRequest} GetKeyspaceRoutingRulesRequest + */ + GetKeyspaceRoutingRulesRequest.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetKeyspaceRoutingRulesRequest) + return object; + return new $root.vtctldata.GetKeyspaceRoutingRulesRequest(); + }; + + /** + * Creates a plain object from a GetKeyspaceRoutingRulesRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {vtctldata.GetKeyspaceRoutingRulesRequest} message GetKeyspaceRoutingRulesRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + GetKeyspaceRoutingRulesRequest.toObject = function toObject() { + return {}; + }; + + /** + * Converts this GetKeyspaceRoutingRulesRequest to JSON. + * @function toJSON + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @instance + * @returns {Object.} JSON object + */ + GetKeyspaceRoutingRulesRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for GetKeyspaceRoutingRulesRequest + * @function getTypeUrl + * @memberof vtctldata.GetKeyspaceRoutingRulesRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + GetKeyspaceRoutingRulesRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.GetKeyspaceRoutingRulesRequest"; + }; + + return GetKeyspaceRoutingRulesRequest; + })(); + + vtctldata.GetKeyspaceRoutingRulesResponse = (function() { + + /** + * Properties of a GetKeyspaceRoutingRulesResponse. + * @memberof vtctldata + * @interface IGetKeyspaceRoutingRulesResponse + * @property {vschema.IKeyspaceRoutingRules|null} [keyspace_routing_rules] GetKeyspaceRoutingRulesResponse keyspace_routing_rules + */ + + /** + * Constructs a new GetKeyspaceRoutingRulesResponse. + * @memberof vtctldata + * @classdesc Represents a GetKeyspaceRoutingRulesResponse. + * @implements IGetKeyspaceRoutingRulesResponse + * @constructor + * @param {vtctldata.IGetKeyspaceRoutingRulesResponse=} [properties] Properties to set + */ + function GetKeyspaceRoutingRulesResponse(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]]; + } + + /** + * GetKeyspaceRoutingRulesResponse keyspace_routing_rules. + * @member {vschema.IKeyspaceRoutingRules|null|undefined} keyspace_routing_rules + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @instance + */ + GetKeyspaceRoutingRulesResponse.prototype.keyspace_routing_rules = null; + + /** + * Creates a new GetKeyspaceRoutingRulesResponse instance using the specified properties. + * @function create + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.IGetKeyspaceRoutingRulesResponse=} [properties] Properties to set + * @returns {vtctldata.GetKeyspaceRoutingRulesResponse} GetKeyspaceRoutingRulesResponse instance + */ + GetKeyspaceRoutingRulesResponse.create = function create(properties) { + return new GetKeyspaceRoutingRulesResponse(properties); + }; + + /** + * Encodes the specified GetKeyspaceRoutingRulesResponse message. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesResponse.verify|verify} messages. + * @function encode + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.IGetKeyspaceRoutingRulesResponse} message GetKeyspaceRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + GetKeyspaceRoutingRulesResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace_routing_rules != null && Object.hasOwnProperty.call(message, "keyspace_routing_rules")) + $root.vschema.KeyspaceRoutingRules.encode(message.keyspace_routing_rules, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified GetKeyspaceRoutingRulesResponse message, length delimited. Does not implicitly {@link vtctldata.GetKeyspaceRoutingRulesResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.IGetKeyspaceRoutingRulesResponse} message GetKeyspaceRoutingRulesResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + GetKeyspaceRoutingRulesResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a GetKeyspaceRoutingRulesResponse message from the specified reader or buffer. + * @function decode + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {vtctldata.GetKeyspaceRoutingRulesResponse} GetKeyspaceRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + GetKeyspaceRoutingRulesResponse.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.vtctldata.GetKeyspaceRoutingRulesResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a GetKeyspaceRoutingRulesResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {vtctldata.GetKeyspaceRoutingRulesResponse} GetKeyspaceRoutingRulesResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + GetKeyspaceRoutingRulesResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a GetKeyspaceRoutingRulesResponse message. + * @function verify + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + GetKeyspaceRoutingRulesResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace_routing_rules != null && message.hasOwnProperty("keyspace_routing_rules")) { + let error = $root.vschema.KeyspaceRoutingRules.verify(message.keyspace_routing_rules); + if (error) + return "keyspace_routing_rules." + error; + } + return null; + }; + + /** + * Creates a GetKeyspaceRoutingRulesResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {Object.} object Plain object + * @returns {vtctldata.GetKeyspaceRoutingRulesResponse} GetKeyspaceRoutingRulesResponse + */ + GetKeyspaceRoutingRulesResponse.fromObject = function fromObject(object) { + if (object instanceof $root.vtctldata.GetKeyspaceRoutingRulesResponse) + return object; + let message = new $root.vtctldata.GetKeyspaceRoutingRulesResponse(); + if (object.keyspace_routing_rules != null) { + if (typeof object.keyspace_routing_rules !== "object") + throw TypeError(".vtctldata.GetKeyspaceRoutingRulesResponse.keyspace_routing_rules: object expected"); + message.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.fromObject(object.keyspace_routing_rules); + } + return message; + }; + + /** + * Creates a plain object from a GetKeyspaceRoutingRulesResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {vtctldata.GetKeyspaceRoutingRulesResponse} message GetKeyspaceRoutingRulesResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + GetKeyspaceRoutingRulesResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.keyspace_routing_rules = null; + if (message.keyspace_routing_rules != null && message.hasOwnProperty("keyspace_routing_rules")) + object.keyspace_routing_rules = $root.vschema.KeyspaceRoutingRules.toObject(message.keyspace_routing_rules, options); + return object; + }; + + /** + * Converts this GetKeyspaceRoutingRulesResponse to JSON. + * @function toJSON + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @instance + * @returns {Object.} JSON object + */ + GetKeyspaceRoutingRulesResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for GetKeyspaceRoutingRulesResponse + * @function getTypeUrl + * @memberof vtctldata.GetKeyspaceRoutingRulesResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + GetKeyspaceRoutingRulesResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/vtctldata.GetKeyspaceRoutingRulesResponse"; + }; + + return GetKeyspaceRoutingRulesResponse; + })(); + vtctldata.GetRoutingRulesRequest = (function() { /** @@ -144392,6 +146469,7 @@ export const vtctldata = $root.vtctldata = (() => { * @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 + * @property {vtctldata.IWorkflowOptions|null} [workflow_options] MoveTablesCreateRequest workflow_options */ /** @@ -144566,6 +146644,14 @@ export const vtctldata = $root.vtctldata = (() => { */ MoveTablesCreateRequest.prototype.atomic_copy = false; + /** + * MoveTablesCreateRequest workflow_options. + * @member {vtctldata.IWorkflowOptions|null|undefined} workflow_options + * @memberof vtctldata.MoveTablesCreateRequest + * @instance + */ + MoveTablesCreateRequest.prototype.workflow_options = null; + /** * Creates a new MoveTablesCreateRequest instance using the specified properties. * @function create @@ -144636,6 +146722,8 @@ export const vtctldata = $root.vtctldata = (() => { 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); + if (message.workflow_options != null && Object.hasOwnProperty.call(message, "workflow_options")) + $root.vtctldata.WorkflowOptions.encode(message.workflow_options, writer.uint32(/* id 20, wireType 2 =*/162).fork()).ldelim(); return writer; }; @@ -144761,6 +146849,10 @@ export const vtctldata = $root.vtctldata = (() => { message.atomic_copy = reader.bool(); break; } + case 20: { + message.workflow_options = $root.vtctldata.WorkflowOptions.decode(reader, reader.uint32()); + break; + } default: reader.skipType(tag & 7); break; @@ -144893,6 +146985,11 @@ export const vtctldata = $root.vtctldata = (() => { if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) if (typeof message.atomic_copy !== "boolean") return "atomic_copy: boolean expected"; + if (message.workflow_options != null && message.hasOwnProperty("workflow_options")) { + let error = $root.vtctldata.WorkflowOptions.verify(message.workflow_options); + if (error) + return "workflow_options." + error; + } return null; }; @@ -145039,6 +147136,11 @@ export const vtctldata = $root.vtctldata = (() => { message.no_routing_rules = Boolean(object.no_routing_rules); if (object.atomic_copy != null) message.atomic_copy = Boolean(object.atomic_copy); + if (object.workflow_options != null) { + if (typeof object.workflow_options !== "object") + throw TypeError(".vtctldata.MoveTablesCreateRequest.workflow_options: object expected"); + message.workflow_options = $root.vtctldata.WorkflowOptions.fromObject(object.workflow_options); + } return message; }; @@ -145077,6 +147179,7 @@ export const vtctldata = $root.vtctldata = (() => { object.auto_start = false; object.no_routing_rules = false; object.atomic_copy = false; + object.workflow_options = null; } if (message.workflow != null && message.hasOwnProperty("workflow")) object.workflow = message.workflow; @@ -145131,6 +147234,8 @@ export const vtctldata = $root.vtctldata = (() => { object.no_routing_rules = message.no_routing_rules; if (message.atomic_copy != null && message.hasOwnProperty("atomic_copy")) object.atomic_copy = message.atomic_copy; + if (message.workflow_options != null && message.hasOwnProperty("workflow_options")) + object.workflow_options = $root.vtctldata.WorkflowOptions.toObject(message.workflow_options, options); return object; };